From d592d27f43906e0ba910435902990719e1cb2eef Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Tue, 12 Sep 2023 15:33:58 +0200 Subject: [PATCH 01/15] Flink: Emitt watermarks from the IcebergSource --- .../iceberg/flink/source/IcebergSource.java | 49 ++- .../EventTimeExtractorRecordEmitter.java | 77 ++++ ...ventTimeExtractorRecordEmitterFactory.java | 39 ++ .../IcebergEventTimeExtractor.java | 30 ++ .../IcebergTimestampEventTimeExtractor.java | 72 ++++ .../source/reader/IcebergSourceReader.java | 3 +- .../reader/IcebergSourceRecordEmitter.java | 1 + .../IcebergSourceRecordEmitterFactory.java | 30 ++ .../source/reader/RecordEmitterFactory.java | 30 ++ .../flink/source/split/SplitComparators.java | 19 +- ...ebergSourceFailoverEventTimeExtractor.java | 116 ++++++ ...stIcebergSourceWithEventTimeExtractor.java | 376 ++++++++++++++++++ .../reader/TestIcebergSourceReader.java | 4 +- 13 files changed, 840 insertions(+), 6 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index f85f6277263b..2c3697799516 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -58,14 +58,19 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.eventtimeextractor.EventTimeExtractorRecordEmitterFactory; +import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; +import org.apache.iceberg.flink.source.reader.IcebergSourceRecordEmitterFactory; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RecordEmitterFactory; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -80,6 +85,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; + private final RecordEmitterFactory emitterFactory; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -91,13 +97,15 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, SerializableComparator splitComparator, - Table table) { + Table table, + RecordEmitterFactory emitterFactory) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; this.table = table; + this.emitterFactory = emitterFactory; } String name() { @@ -152,7 +160,8 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + emitterFactory, metrics, readerFunction, splitComparator, readerContext); } @Override @@ -216,6 +225,8 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; + private IcebergEventTimeExtractor timeExtractor; + private RecordEmitterFactory emitterFactory; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -237,6 +248,9 @@ public Builder table(Table newTable) { } public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + Preconditions.checkArgument( + timeExtractor == null, + "TimestampAssigner and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -429,6 +443,20 @@ public Builder setAll(Map properties) { return this; } + /** + * Sets the {@link IcebergEventTimeExtractor} to retrieve the split watermark and the record + * timestamps when emitting the records. The {@link + * IcebergEventTimeExtractor#extractWatermark(IcebergSourceSplit)} is also used for ordering the + * splits for read. + */ + public Builder eventTimeExtractor(IcebergEventTimeExtractor newTimeExtractor) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "TimestampAssigner and SplitAssigner should not be set in the same source"); + this.timeExtractor = newTimeExtractor; + return this; + } + /** @deprecated Use {@link #setAll} instead. */ @Deprecated public Builder properties(Map properties) { @@ -483,15 +511,30 @@ public IcebergSource build() { } } + if (timeExtractor == null) { + emitterFactory = new IcebergSourceRecordEmitterFactory<>(); + } else { + emitterFactory = new EventTimeExtractorRecordEmitterFactory<>(timeExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermarkComparator(timeExtractor)); + } + checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitterFactory); } private void checkRequired() { Preconditions.checkNotNull(tableLoader, "tableLoader is required."); Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(emitterFactory, "emitterFactory is required."); Preconditions.checkNotNull(readerFunction, "readerFunction is required."); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java new file mode 100644 index 000000000000..56204d41642d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java @@ -0,0 +1,77 @@ +/* + * 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.eventtimeextractor; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.reader.RecordAndPosition; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the record with event time and updates the split position. + * + *

The Emitter also emits watermarks at the beginning of every split, and sets the event + * timestamp based on the provided {@link IcebergEventTimeExtractor}. + */ +final class EventTimeExtractorRecordEmitter + implements RecordEmitter, T, IcebergSourceSplit> { + private static final Logger LOG = LoggerFactory.getLogger(EventTimeExtractorRecordEmitter.class); + private final IcebergEventTimeExtractor timeExtractor; + private String lastSplit = null; + private long watermark; + + EventTimeExtractorRecordEmitter(IcebergEventTimeExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplit)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.warn( + "Watermark decreased. PreviousWM {}, currentWM {}, previousSplit {}, currentSplit {}.", + watermark, + newWatermark, + lastSplit, + split.splitId()); + } + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + lastSplit = split.splitId(); + } + + long eventTime = timeExtractor.extractEventTime(element.record()); + if (eventTime <= watermark) { + LOG.warn( + "Late event arrived. PreviousWM {}, split {}, eventTime {}, record {}.", + watermark, + split, + eventTime, + element.record()); + } + + output.collect(element.record(), eventTime); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java new file mode 100644 index 000000000000..46306af049ba --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java @@ -0,0 +1,39 @@ +/* + * 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.eventtimeextractor; + +import java.io.Serializable; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.reader.RecordAndPosition; +import org.apache.iceberg.flink.source.reader.RecordEmitterFactory; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +public class EventTimeExtractorRecordEmitterFactory + implements RecordEmitterFactory, Serializable { + private final IcebergEventTimeExtractor extractor; + + public EventTimeExtractorRecordEmitterFactory(IcebergEventTimeExtractor extractor) { + this.extractor = extractor; + } + + @Override + public RecordEmitter, T, IcebergSourceSplit> emitter() { + return new EventTimeExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java new file mode 100644 index 000000000000..b75ef94b1f58 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java @@ -0,0 +1,30 @@ +/* + * 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.eventtimeextractor; + +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** The interface used to extract watermarks and event timestamps from splits and records. */ +public interface IcebergEventTimeExtractor { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); + + /** Get the event timestamp for the record. */ + long extractEventTime(T record); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java new file mode 100644 index 000000000000..dd63c1174d06 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java @@ -0,0 +1,72 @@ +/* + * 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.eventtimeextractor; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +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 IcebergEventTimeExtractor} implementation which uses an Iceberg timestamp column to get + * the watermarks and the event times for the {@link RowData} read by the reader function. + */ +public class IcebergTimestampEventTimeExtractor + implements IcebergEventTimeExtractor, Serializable { + private final int tsFieldId; + private final int tsFiledPos; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param tsFieldName The timestamp column which should be used as an event time + */ + public IcebergTimestampEventTimeExtractor(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(); + this.tsFiledPos = FlinkSchemaUtil.convert(schema).getFieldIndex(tsFieldName); + } + + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> + (long) + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)) + / 1000L) + .min(Comparator.comparingLong(l -> l)) + .get(); + } + + @Override + public long extractEventTime(RowData rowData) { + return rowData.getTimestamp(tsFiledPos, 0).getMillisecond(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 8d7d68f961cb..84ed9b91d23b 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -35,13 +35,14 @@ public class IcebergSourceReader RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { public IcebergSourceReader( + RecordEmitterFactory emitterFactory, IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, SerializableComparator splitComparator, SourceReaderContext context) { super( () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - new IcebergSourceRecordEmitter<>(), + emitterFactory.emitter(), context.getConfiguration(), context); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java index 337d9d3c4223..0078847047ab 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java @@ -22,6 +22,7 @@ import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +/** Simple emitter which emits the record and updates the split position. */ final class IcebergSourceRecordEmitter implements RecordEmitter, T, IcebergSourceSplit> { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java new file mode 100644 index 000000000000..483da23f7c25 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java @@ -0,0 +1,30 @@ +/* + * 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 org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +public class IcebergSourceRecordEmitterFactory implements RecordEmitterFactory, Serializable { + @Override + public RecordEmitter, T, IcebergSourceSplit> emitter() { + return new IcebergSourceRecordEmitter<>(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java new file mode 100644 index 000000000000..b139146548a7 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java @@ -0,0 +1,30 @@ +/* + * 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.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** + * Factory defining which {@link org.apache.flink.connector.base.source.reader.RecordEmitter} + * implementation to use. + */ +public interface RecordEmitterFactory { + RecordEmitter, T, IcebergSourceSplit> emitter(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 64e03d77debe..b8fa0a20cbf5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -45,7 +46,7 @@ public static SerializableComparator fileSequenceNumber() { o1); Preconditions.checkNotNull( seq2, - "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", o2); int temp = Long.compare(seq1, seq2); @@ -56,4 +57,20 @@ public static SerializableComparator fileSequenceNumber() { } }; } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermarkComparator( + IcebergEventTimeExtractor eventTimeExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long seq1 = eventTimeExtractor.extractWatermark(o1); + long seq2 = eventTimeExtractor.extractWatermark(o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java new file mode 100644 index 000000000000..c669ebfe7748 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java @@ -0,0 +1,116 @@ +/* + * 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; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.eventtimeextractor.IcebergTimestampEventTimeExtractor; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; + +public class TestIcebergSourceFailoverEventTimeExtractor extends TestIcebergSourceFailover { + // increment ts by 60 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 minute + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .eventTimeExtractor(new IcebergTimestampEventTimeExtractor(TestFixtures.TS_SCHEMA, "ts")) + .project(TestFixtures.TS_SCHEMA) + .includeColumnStats(true); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .map( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + return record; + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords( + Table table, List expectedRecords, Duration interval, int maxCount) throws Exception { + List expectedNormalized = convertTimestampField(expectedRecords); + for (int i = 0; i < maxCount; ++i) { + if (SimpleDataUtil.equalsRecords( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema())) { + break; + } else { + Thread.sleep(interval.toMillis()); + } + } + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema()); + } + + private List convertTimestampField(List records) { + return records.stream() + .map( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + return r; + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java new file mode 100644 index 000000000000..e473c8ac89c5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java @@ -0,0 +1,376 @@ +/* + * 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; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; +import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.eventtimeextractor.IcebergTimestampEventTimeExtractor; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithEventTimeExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "File-1-100"), (103, "File-1-103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + // - Split 2 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + // - File 3 - Parallel write for the first records (Watermark 60000) + // - Split 1 - 2 records (1, "File-3-1"), (3, "File-3-3") + List batch; + batch = ImmutableList.of(generateRecord(100, "100"), generateRecord(103, "103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(i % 5, "File-2-" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = ImmutableList.of(generateRecord(1, "File-3-1"), generateRecord(3, "File-3-3")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.noWatermarks(), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Just print all the data to confirm everything has arrived + values.forEach(r -> out.collect(r)); + } + }); + + CollectResultIterator resultIterator = addCollectSink(windowed); + + // Start the job + JobClient jobClient = env.executeAsync("Iceberg Source Windowing Test"); + resultIterator.setJobClient(jobClient); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + + assertRecords(resultIterator, expectedRecords); + } + + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "File-1-100"), (103, "File-1-103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + // - Split 2 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + List batch; + batch = ImmutableList.of(generateRecord(100, "File-1-100"), generateRecord(103, "File-1-103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(i % 5, "File-2-" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + CollectResultIterator resultIterator = addCollectSink(stream); + + // Start the job + JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + resultIterator.setJobClient(jobClient); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); + Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + batch = + ImmutableList.of( + generateRecord(15, "File-3-15"), + generateRecord(16, "File-3-16"), + generateRecord(17, "File-3-17")); + dataAppender.appendToTable(batch); + batch = + ImmutableList.of( + generateRecord(15, "File-4-15"), + generateRecord(16, "File-4-16"), + generateRecord(17, "File-4-17")); + dataAppender.appendToTable(batch); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - File-1) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); + + // Add some new records which should unblock the throttled reader + batch = ImmutableList.of(generateRecord(110, "File-5-110"), generateRecord(111, "File-5-111")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); + } + + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .eventTimeExtractor(new IcebergTimestampEventTimeExtractor(TestFixtures.TS_SCHEMA, "ts")) + .project(TestFixtures.TS_SCHEMA) + .includeColumnStats(true) + .splitSize(100L); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + protected void assertRecords( + CollectResultIterator iterator, List expectedRecords) throws Exception { + + Set received = Sets.newHashSetWithExpectedSize(expectedRecords.size()); + + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < expectedRecords.size() && iterator.hasNext()) { + received.add(iterator.next()); + count++; + } + if (count < expectedRecords.size()) { + throw new IllegalStateException( + String.format("Fail to get %d records.", expectedRecords.size())); + } + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + Set expected = + expectedRecords.stream() + .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) + .collect(Collectors.toSet()); + Assert.assertEquals(expected, received); + } + + protected void waitForRecords(CollectResultIterator iterator, int num) throws Exception { + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < num && iterator.hasNext()) { + iterator.next(); + count++; + } + if (count < num) { + throw new IllegalStateException(String.format("Fail to get %d records.", num)); + } + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + } + + private CollectResultIterator addCollectSink(DataStream stream) { + TypeSerializer serializer = + stream.getType().createSerializer(stream.getExecutionConfig()); + String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); + CollectSinkOperatorFactory factory = + new CollectSinkOperatorFactory<>(serializer, accumulatorName); + CollectSinkOperator operator = (CollectSinkOperator) factory.getOperator(); + CollectStreamSink sink = new CollectStreamSink<>(stream, factory); + sink.name("Data stream collect sink"); + stream.getExecutionEnvironment().addOperator(sink.getTransformation()); + return new CollectResultIterator<>( + operator.getOperatorIdFuture(), + serializer, + accumulatorName, + stream.getExecutionEnvironment().getCheckpointConfig()); + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index def4f436851b..f403b64ed879 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -167,7 +167,9 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + RecordEmitterFactory emitterFactory = new IcebergSourceRecordEmitterFactory<>(); + return new IcebergSourceReader<>( + emitterFactory, readerMetrics, readerFunction, splitComparator, readerContext); } private static class IdBasedComparator implements SerializableComparator { From b2ff85fbcbb11d56f82690debd0a203ed9817f71 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 13 Sep 2023 12:27:48 +0200 Subject: [PATCH 02/15] Fix review comments --- .../iceberg/flink/source/IcebergSource.java | 17 +++++++++-------- .../IcebergEventTimeExtractor.java | 3 ++- .../IcebergTimestampEventTimeExtractor.java | 6 +++--- .../source/reader/RecordEmitterFactory.java | 3 ++- .../flink/source/split/SplitComparators.java | 6 +++--- ...IcebergSourceFailoverEventTimeExtractor.java | 2 +- 6 files changed, 20 insertions(+), 17 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 2c3697799516..5e99d0ce0df5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -225,7 +225,7 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; - private IcebergEventTimeExtractor timeExtractor; + private IcebergEventTimeExtractor eventTimeExtractor; private RecordEmitterFactory emitterFactory; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); @@ -249,7 +249,7 @@ public Builder table(Table newTable) { public Builder assignerFactory(SplitAssignerFactory assignerFactory) { Preconditions.checkArgument( - timeExtractor == null, + eventTimeExtractor == null, "TimestampAssigner and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; @@ -449,11 +449,11 @@ public Builder setAll(Map properties) { * IcebergEventTimeExtractor#extractWatermark(IcebergSourceSplit)} is also used for ordering the * splits for read. */ - public Builder eventTimeExtractor(IcebergEventTimeExtractor newTimeExtractor) { + public Builder eventTimeExtractor(IcebergEventTimeExtractor newEventTimeExtractor) { Preconditions.checkArgument( splitAssignerFactory == null, "TimestampAssigner and SplitAssigner should not be set in the same source"); - this.timeExtractor = newTimeExtractor; + this.eventTimeExtractor = newEventTimeExtractor; return this; } @@ -511,17 +511,18 @@ public IcebergSource build() { } } - if (timeExtractor == null) { + if (eventTimeExtractor == null) { emitterFactory = new IcebergSourceRecordEmitterFactory<>(); } else { - emitterFactory = new EventTimeExtractorRecordEmitterFactory<>(timeExtractor); + emitterFactory = new EventTimeExtractorRecordEmitterFactory<>(eventTimeExtractor); splitAssignerFactory = - new OrderedSplitAssignerFactory(SplitComparators.watermarkComparator(timeExtractor)); + new OrderedSplitAssignerFactory( + SplitComparators.watermarkComparator(eventTimeExtractor)); } checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource( + return new IcebergSource<>( tableLoader, context, readerFunction, diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java index b75ef94b1f58..51d258618f96 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.flink.source.eventtimeextractor; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; /** The interface used to extract watermarks and event timestamps from splits and records. */ -public interface IcebergEventTimeExtractor { +public interface IcebergEventTimeExtractor extends Serializable { /** Get the watermark for a split. */ long extractWatermark(IcebergSourceSplit split); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java index dd63c1174d06..6695388aef0a 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java @@ -36,7 +36,7 @@ public class IcebergTimestampEventTimeExtractor implements IcebergEventTimeExtractor, Serializable { private final int tsFieldId; - private final int tsFiledPos; + private final int tsFieldPos; /** * Creates the extractor. @@ -49,7 +49,7 @@ public IcebergTimestampEventTimeExtractor(Schema schema, String tsFieldName) { Preconditions.checkArgument( field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); this.tsFieldId = field.fieldId(); - this.tsFiledPos = FlinkSchemaUtil.convert(schema).getFieldIndex(tsFieldName); + this.tsFieldPos = FlinkSchemaUtil.convert(schema).getFieldIndex(tsFieldName); } @Override @@ -67,6 +67,6 @@ public long extractWatermark(IcebergSourceSplit split) { @Override public long extractEventTime(RowData rowData) { - return rowData.getTimestamp(tsFiledPos, 0).getMillisecond(); + return rowData.getTimestamp(tsFieldPos, 0).getMillisecond(); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java index b139146548a7..53642da46f79 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.io.Serializable; import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -25,6 +26,6 @@ * Factory defining which {@link org.apache.flink.connector.base.source.reader.RecordEmitter} * implementation to use. */ -public interface RecordEmitterFactory { +public interface RecordEmitterFactory extends Serializable { RecordEmitter, T, IcebergSourceSplit> emitter(); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index b8fa0a20cbf5..0d56d861fa79 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -62,10 +62,10 @@ public static SerializableComparator fileSequenceNumber() { public static SerializableComparator watermarkComparator( IcebergEventTimeExtractor eventTimeExtractor) { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - long seq1 = eventTimeExtractor.extractWatermark(o1); - long seq2 = eventTimeExtractor.extractWatermark(o2); + long watermark1 = eventTimeExtractor.extractWatermark(o1); + long watermark2 = eventTimeExtractor.extractWatermark(o2); - int temp = Long.compare(seq1, seq2); + int temp = Long.compare(watermark1, watermark2); if (temp != 0) { return temp; } else { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java index c669ebfe7748..e460b9d5e820 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java @@ -39,7 +39,7 @@ import org.apache.iceberg.util.StructLikeWrapper; public class TestIcebergSourceFailoverEventTimeExtractor extends TestIcebergSourceFailover { - // increment ts by 60 minutes for each generateRecords batch + // Increment ts by 60 minutes for each generateRecords batch private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); // Within a batch, increment ts by 1 minute private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); From 7771586de25e150c77fc1929504c665e0a354c0a Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Fri, 15 Sep 2023 12:13:57 +0200 Subject: [PATCH 03/15] Use RecordEmmitters instead of factories to simplify the code --- .../iceberg/flink/source/IcebergSource.java | 22 +++++---- .../EventTimeExtractorRecordEmitter.java | 7 ++- ...ventTimeExtractorRecordEmitterFactory.java | 39 ---------------- .../source/reader/IcebergSourceReader.java | 4 +- .../reader/IcebergSourceRecordEmitter.java | 37 --------------- .../flink/source/reader/RecordEmmitters.java | 45 +++++++++++++++++++ ...ry.java => SerializableRecordEmitter.java} | 8 +--- .../reader/TestIcebergSourceReader.java | 3 +- 8 files changed, 63 insertions(+), 102 deletions(-) delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergSourceRecordEmitterFactory.java => SerializableRecordEmitter.java} (80%) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 5e99d0ce0df5..bc5a207d975d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -58,15 +58,14 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; -import org.apache.iceberg.flink.source.eventtimeextractor.EventTimeExtractorRecordEmitterFactory; import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.IcebergSourceRecordEmitterFactory; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RecordEmitterFactory; +import org.apache.iceberg.flink.source.reader.RecordEmmitters; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; @@ -85,7 +84,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; - private final RecordEmitterFactory emitterFactory; + private final SerializableRecordEmitter emitter; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -98,14 +97,14 @@ public class IcebergSource implements Source splitComparator, Table table, - RecordEmitterFactory emitterFactory) { + SerializableRecordEmitter emitter) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; this.table = table; - this.emitterFactory = emitterFactory; + this.emitter = emitter; } String name() { @@ -161,7 +160,7 @@ public SourceReader createReader(SourceReaderContext read IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); return new IcebergSourceReader<>( - emitterFactory, metrics, readerFunction, splitComparator, readerContext); + emitter, metrics, readerFunction, splitComparator, readerContext); } @Override @@ -226,7 +225,6 @@ public static class Builder { private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private IcebergEventTimeExtractor eventTimeExtractor; - private RecordEmitterFactory emitterFactory; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -511,10 +509,11 @@ public IcebergSource build() { } } + SerializableRecordEmitter emitter; if (eventTimeExtractor == null) { - emitterFactory = new IcebergSourceRecordEmitterFactory<>(); + emitter = RecordEmmitters.emitter(); } else { - emitterFactory = new EventTimeExtractorRecordEmitterFactory<>(eventTimeExtractor); + emitter = RecordEmmitters.emitter(eventTimeExtractor); splitAssignerFactory = new OrderedSplitAssignerFactory( SplitComparators.watermarkComparator(eventTimeExtractor)); @@ -529,13 +528,12 @@ public IcebergSource build() { splitAssignerFactory, splitComparator, table, - emitterFactory); + emitter); } private void checkRequired() { Preconditions.checkNotNull(tableLoader, "tableLoader is required."); Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(emitterFactory, "emitterFactory is required."); Preconditions.checkNotNull(readerFunction, "readerFunction is required."); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java index 56204d41642d..5e9606aab1ce 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.reader.RecordAndPosition; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,14 +32,13 @@ *

The Emitter also emits watermarks at the beginning of every split, and sets the event * timestamp based on the provided {@link IcebergEventTimeExtractor}. */ -final class EventTimeExtractorRecordEmitter - implements RecordEmitter, T, IcebergSourceSplit> { +public final class EventTimeExtractorRecordEmitter implements SerializableRecordEmitter { private static final Logger LOG = LoggerFactory.getLogger(EventTimeExtractorRecordEmitter.class); private final IcebergEventTimeExtractor timeExtractor; private String lastSplit = null; private long watermark; - EventTimeExtractorRecordEmitter(IcebergEventTimeExtractor timeExtractor) { + public EventTimeExtractorRecordEmitter(IcebergEventTimeExtractor timeExtractor) { this.timeExtractor = timeExtractor; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java deleted file mode 100644 index 46306af049ba..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitterFactory.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.eventtimeextractor; - -import java.io.Serializable; -import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.reader.RecordAndPosition; -import org.apache.iceberg.flink.source.reader.RecordEmitterFactory; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -public class EventTimeExtractorRecordEmitterFactory - implements RecordEmitterFactory, Serializable { - private final IcebergEventTimeExtractor extractor; - - public EventTimeExtractorRecordEmitterFactory(IcebergEventTimeExtractor extractor) { - this.extractor = extractor; - } - - @Override - public RecordEmitter, T, IcebergSourceSplit> emitter() { - return new EventTimeExtractorRecordEmitter<>(extractor); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 84ed9b91d23b..f143b8d2df2e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -35,14 +35,14 @@ public class IcebergSourceReader RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { public IcebergSourceReader( - RecordEmitterFactory emitterFactory, + SerializableRecordEmitter emitter, IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, SerializableComparator splitComparator, SourceReaderContext context) { super( () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - emitterFactory.emitter(), + emitter, context.getConfiguration(), context); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java deleted file mode 100644 index 0078847047ab..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** Simple emitter which emits the record and updates the split position. */ -final class IcebergSourceRecordEmitter - implements RecordEmitter, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java new file mode 100644 index 000000000000..6b884330f80d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java @@ -0,0 +1,45 @@ +/* + * 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.iceberg.flink.source.eventtimeextractor.EventTimeExtractorRecordEmitter; +import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Provides implementations of {@link SerializableRecordEmitter} which could be used for emitting + * records from an Iceberg split. These are used by the {@link IcebergSourceReader} + */ +public class RecordEmmitters { + private static final Logger LOG = LoggerFactory.getLogger(RecordEmmitters.class); + + private RecordEmmitters() {} + + public static SerializableRecordEmitter emitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + public static SerializableRecordEmitter emitter(IcebergEventTimeExtractor extractor) { + return new EventTimeExtractorRecordEmitter(extractor); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 80% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index 483da23f7c25..a82d212a441d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitterFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -22,9 +22,5 @@ import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -public class IcebergSourceRecordEmitterFactory implements RecordEmitterFactory, Serializable { - @Override - public RecordEmitter, T, IcebergSourceSplit> emitter() { - return new IcebergSourceRecordEmitter<>(); - } -} +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable {} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index f403b64ed879..5ad25c385446 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -167,9 +167,8 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - RecordEmitterFactory emitterFactory = new IcebergSourceRecordEmitterFactory<>(); return new IcebergSourceReader<>( - emitterFactory, readerMetrics, readerFunction, splitComparator, readerContext); + RecordEmmitters.emitter(), readerMetrics, readerFunction, splitComparator, readerContext); } private static class IdBasedComparator implements SerializableComparator { From cf8e080db5827cbfb55ce005ed81286401be21a9 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Fri, 15 Sep 2023 21:04:20 +0200 Subject: [PATCH 04/15] Fix the emmitter --- .../java/org/apache/iceberg/flink/source/IcebergSource.java | 6 +++--- .../reader/{RecordEmmitters.java => RecordEmitters.java} | 6 +++--- .../flink/source/reader/TestIcebergSourceReader.java | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{RecordEmmitters.java => RecordEmitters.java} (95%) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index bc5a207d975d..30f85077032a 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -63,7 +63,7 @@ import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RecordEmmitters; +import org.apache.iceberg.flink.source.reader.RecordEmitters; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -511,9 +511,9 @@ public IcebergSource build() { SerializableRecordEmitter emitter; if (eventTimeExtractor == null) { - emitter = RecordEmmitters.emitter(); + emitter = RecordEmitters.emitter(); } else { - emitter = RecordEmmitters.emitter(eventTimeExtractor); + emitter = RecordEmitters.emitter(eventTimeExtractor); splitAssignerFactory = new OrderedSplitAssignerFactory( SplitComparators.watermarkComparator(eventTimeExtractor)); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java similarity index 95% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java index 6b884330f80d..c7ea73d8609f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmmitters.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java @@ -27,10 +27,10 @@ * Provides implementations of {@link SerializableRecordEmitter} which could be used for emitting * records from an Iceberg split. These are used by the {@link IcebergSourceReader} */ -public class RecordEmmitters { - private static final Logger LOG = LoggerFactory.getLogger(RecordEmmitters.class); +public class RecordEmitters { + private static final Logger LOG = LoggerFactory.getLogger(RecordEmitters.class); - private RecordEmmitters() {} + private RecordEmitters() {} public static SerializableRecordEmitter emitter() { return (element, output, split) -> { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 5ad25c385446..93b083fcddff 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -168,7 +168,7 @@ private IcebergSourceReader createReader( new PlaintextEncryptionManager(), Collections.emptyList()); return new IcebergSourceReader<>( - RecordEmmitters.emitter(), readerMetrics, readerFunction, splitComparator, readerContext); + RecordEmitters.emitter(), readerMetrics, readerFunction, splitComparator, readerContext); } private static class IdBasedComparator implements SerializableComparator { From d5226a7a7183bf9e1268d7c710f69578f26e962f Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 27 Sep 2023 17:55:07 +0200 Subject: [PATCH 05/15] Steven's comments --- .../iceberg/flink/source/IcebergSource.java | 29 ++++++------ .../IcebergTimestampWatermarkExtractor.java} | 21 +++------ .../IcebergWatermarkExtractor.java} | 9 ++-- .../source/reader/RecordEmitterFactory.java | 31 ------------- .../flink/source/reader/RecordEmitters.java | 45 ------------------- .../reader/SerializableRecordEmitter.java | 14 +++++- .../WatermarkExtractorRecordEmitter.java} | 30 ++++--------- .../flink/source/split/SplitComparators.java | 8 ++-- ...SourceFailoverWithWatermarkExtractor.java} | 8 ++-- ...tIcebergSourceWithWatermarkExtractor.java} | 9 ++-- .../reader/TestIcebergSourceReader.java | 6 ++- 11 files changed, 64 insertions(+), 146 deletions(-) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/{eventtimeextractor/IcebergTimestampEventTimeExtractor.java => reader/IcebergTimestampWatermarkExtractor.java} (74%) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/{eventtimeextractor/IcebergEventTimeExtractor.java => reader/IcebergWatermarkExtractor.java} (76%) delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/{eventtimeextractor/EventTimeExtractorRecordEmitter.java => reader/WatermarkExtractorRecordEmitter.java} (62%) rename flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/{TestIcebergSourceFailoverEventTimeExtractor.java => TestIcebergSourceFailoverWithWatermarkExtractor.java} (93%) rename flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/{TestIcebergSourceWithEventTimeExtractor.java => TestIcebergSourceWithWatermarkExtractor.java} (98%) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 30f85077032a..5f2aa068a3f7 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -58,12 +58,11 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; -import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; +import org.apache.iceberg.flink.source.reader.IcebergWatermarkExtractor; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RecordEmitters; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -224,7 +223,7 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; - private IcebergEventTimeExtractor eventTimeExtractor; + private IcebergWatermarkExtractor watermarkExtractor; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -247,8 +246,8 @@ public Builder table(Table newTable) { public Builder assignerFactory(SplitAssignerFactory assignerFactory) { Preconditions.checkArgument( - eventTimeExtractor == null, - "TimestampAssigner and SplitAssigner should not be set in the same source"); + watermarkExtractor == null, + "WatermarkExtractor and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -442,16 +441,16 @@ public Builder setAll(Map properties) { } /** - * Sets the {@link IcebergEventTimeExtractor} to retrieve the split watermark and the record - * timestamps when emitting the records. The {@link - * IcebergEventTimeExtractor#extractWatermark(IcebergSourceSplit)} is also used for ordering the + * Sets the {@link IcebergWatermarkExtractor} to retrieve the split watermark before emitting + * the records for a given split. The {@link + * IcebergWatermarkExtractor#extractWatermark(IcebergSourceSplit)} is also used for ordering the * splits for read. */ - public Builder eventTimeExtractor(IcebergEventTimeExtractor newEventTimeExtractor) { + public Builder watermarkExtractor(IcebergWatermarkExtractor newWatermarkExtractor) { Preconditions.checkArgument( splitAssignerFactory == null, - "TimestampAssigner and SplitAssigner should not be set in the same source"); - this.eventTimeExtractor = newEventTimeExtractor; + "WatermarkExtractor and SplitAssigner should not be set in the same source"); + this.watermarkExtractor = newWatermarkExtractor; return this; } @@ -510,13 +509,13 @@ public IcebergSource build() { } SerializableRecordEmitter emitter; - if (eventTimeExtractor == null) { - emitter = RecordEmitters.emitter(); + if (watermarkExtractor == null) { + emitter = SerializableRecordEmitter.defaultEmitter(); } else { - emitter = RecordEmitters.emitter(eventTimeExtractor); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); splitAssignerFactory = new OrderedSplitAssignerFactory( - SplitComparators.watermarkComparator(eventTimeExtractor)); + SplitComparators.watermarkComparator(watermarkExtractor)); } checkRequired(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java similarity index 74% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java index 6695388aef0a..a2039b28e857 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergTimestampEventTimeExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java @@ -16,13 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.eventtimeextractor; +package org.apache.iceberg.flink.source.reader; import java.io.Serializable; import java.util.Comparator; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Conversions; @@ -30,13 +29,13 @@ import org.apache.iceberg.types.Types; /** - * {@link IcebergEventTimeExtractor} implementation which uses an Iceberg timestamp column to get - * the watermarks and the event times for the {@link RowData} read by the reader function. + * {@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 IcebergTimestampEventTimeExtractor - implements IcebergEventTimeExtractor, Serializable { +public class IcebergTimestampWatermarkExtractor + implements IcebergWatermarkExtractor, Serializable { private final int tsFieldId; - private final int tsFieldPos; /** * Creates the extractor. @@ -44,12 +43,11 @@ public class IcebergTimestampEventTimeExtractor * @param schema The schema of the Table * @param tsFieldName The timestamp column which should be used as an event time */ - public IcebergTimestampEventTimeExtractor(Schema schema, String tsFieldName) { + public IcebergTimestampWatermarkExtractor(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(); - this.tsFieldPos = FlinkSchemaUtil.convert(schema).getFieldIndex(tsFieldName); } @Override @@ -64,9 +62,4 @@ public long extractWatermark(IcebergSourceSplit split) { .min(Comparator.comparingLong(l -> l)) .get(); } - - @Override - public long extractEventTime(RowData rowData) { - return rowData.getTimestamp(tsFieldPos, 0).getMillisecond(); - } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java similarity index 76% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java index 51d258618f96..0221e42b6e9e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/IcebergEventTimeExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java @@ -16,16 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.eventtimeextractor; +package org.apache.iceberg.flink.source.reader; import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -/** The interface used to extract watermarks and event timestamps from splits and records. */ -public interface IcebergEventTimeExtractor extends Serializable { +/** The interface used to extract watermarks from splits. */ +public interface IcebergWatermarkExtractor extends Serializable { /** Get the watermark for a split. */ long extractWatermark(IcebergSourceSplit split); - - /** Get the event timestamp for the record. */ - long extractEventTime(T record); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java deleted file mode 100644 index 53642da46f79..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitterFactory.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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 org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** - * Factory defining which {@link org.apache.flink.connector.base.source.reader.RecordEmitter} - * implementation to use. - */ -public interface RecordEmitterFactory extends Serializable { - RecordEmitter, T, IcebergSourceSplit> emitter(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java deleted file mode 100644 index c7ea73d8609f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordEmitters.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.iceberg.flink.source.eventtimeextractor.EventTimeExtractorRecordEmitter; -import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Provides implementations of {@link SerializableRecordEmitter} which could be used for emitting - * records from an Iceberg split. These are used by the {@link IcebergSourceReader} - */ -public class RecordEmitters { - private static final Logger LOG = LoggerFactory.getLogger(RecordEmitters.class); - - private RecordEmitters() {} - - public static SerializableRecordEmitter emitter() { - return (element, output, split) -> { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - }; - } - - public static SerializableRecordEmitter emitter(IcebergEventTimeExtractor extractor) { - return new EventTimeExtractorRecordEmitter(extractor); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index a82d212a441d..c8c8adc0a440 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -23,4 +23,16 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; public interface SerializableRecordEmitter - extends RecordEmitter, T, IcebergSourceSplit>, Serializable {} + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark( + IcebergWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter(extractor); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 62% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index 5e9606aab1ce..2b6e38f8ad39 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/eventtimeextractor/EventTimeExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -16,29 +16,27 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.eventtimeextractor; +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.reader.RecordAndPosition; -import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Emitter which emits the record with event time and updates the split position. + * Emitter which emits the watermarks, records and updates the split position. * - *

The Emitter also emits watermarks at the beginning of every split, and sets the event - * timestamp based on the provided {@link IcebergEventTimeExtractor}. + *

The Emitter emits watermarks at the beginning of every split provided by the {@link + * IcebergWatermarkExtractor}. */ -public final class EventTimeExtractorRecordEmitter implements SerializableRecordEmitter { - private static final Logger LOG = LoggerFactory.getLogger(EventTimeExtractorRecordEmitter.class); - private final IcebergEventTimeExtractor timeExtractor; +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final IcebergWatermarkExtractor timeExtractor; private String lastSplit = null; private long watermark; - public EventTimeExtractorRecordEmitter(IcebergEventTimeExtractor timeExtractor) { + WatermarkExtractorRecordEmitter(IcebergWatermarkExtractor timeExtractor) { this.timeExtractor = timeExtractor; } @@ -60,17 +58,7 @@ public void emitRecord( lastSplit = split.splitId(); } - long eventTime = timeExtractor.extractEventTime(element.record()); - if (eventTime <= watermark) { - LOG.warn( - "Late event arrived. PreviousWM {}, split {}, eventTime {}, record {}.", - watermark, - split, - eventTime, - element.record()); - } - - output.collect(element.record(), eventTime); + output.collect(element.record()); split.updatePosition(element.fileOffset(), element.recordOffset()); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 0d56d861fa79..cda7437e2413 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source.split; -import org.apache.iceberg.flink.source.eventtimeextractor.IcebergEventTimeExtractor; +import org.apache.iceberg.flink.source.reader.IcebergWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -60,10 +60,10 @@ public static SerializableComparator fileSequenceNumber() { /** Comparator which orders the splits based on watermark of the splits */ public static SerializableComparator watermarkComparator( - IcebergEventTimeExtractor eventTimeExtractor) { + IcebergWatermarkExtractor watermarkExtractor) { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - long watermark1 = eventTimeExtractor.extractWatermark(o1); - long watermark2 = eventTimeExtractor.extractWatermark(o2); + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); int temp = Long.compare(watermark1, watermark2); if (temp != 0) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 93% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java rename to flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index e460b9d5e820..50bcacbe2b1d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverEventTimeExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -34,12 +34,12 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.eventtimeextractor.IcebergTimestampEventTimeExtractor; +import org.apache.iceberg.flink.source.reader.IcebergTimestampWatermarkExtractor; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; -public class TestIcebergSourceFailoverEventTimeExtractor extends TestIcebergSourceFailover { - // Increment ts by 60 minutes for each generateRecords batch +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); // Within a batch, increment ts by 1 minute private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); @@ -50,7 +50,7 @@ public class TestIcebergSourceFailoverEventTimeExtractor extends TestIcebergSour protected IcebergSource.Builder sourceBuilder() { return IcebergSource.builder() .tableLoader(sourceTableResource.tableLoader()) - .eventTimeExtractor(new IcebergTimestampEventTimeExtractor(TestFixtures.TS_SCHEMA, "ts")) + .watermarkExtractor(new IcebergTimestampWatermarkExtractor(TestFixtures.TS_SCHEMA, "ts")) .project(TestFixtures.TS_SCHEMA) .includeColumnStats(true); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java rename to flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index e473c8ac89c5..8844ae4e2a1a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithEventTimeExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -66,7 +66,7 @@ import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.RowDataConverter; import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.eventtimeextractor.IcebergTimestampEventTimeExtractor; +import org.apache.iceberg.flink.source.reader.IcebergTimestampWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -79,7 +79,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -public class TestIcebergSourceWithEventTimeExtractor implements Serializable { +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); private static final int PARALLELISM = 4; private static final String SOURCE_NAME = "IcebergSource"; @@ -142,7 +142,8 @@ public void testWindowing() throws Exception { .monitorInterval(Duration.ofMillis(10)) .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(), - WatermarkStrategy.noWatermarks(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), SOURCE_NAME, TypeInformation.of(RowData.class)); DataStream windowed = @@ -264,7 +265,7 @@ public void testThrottling() throws Exception { protected IcebergSource.Builder sourceBuilder() { return IcebergSource.builder() .tableLoader(sourceTableResource.tableLoader()) - .eventTimeExtractor(new IcebergTimestampEventTimeExtractor(TestFixtures.TS_SCHEMA, "ts")) + .watermarkExtractor(new IcebergTimestampWatermarkExtractor(TestFixtures.TS_SCHEMA, "ts")) .project(TestFixtures.TS_SCHEMA) .includeColumnStats(true) .splitSize(100L); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 93b083fcddff..88234c61123f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -168,7 +168,11 @@ private IcebergSourceReader createReader( new PlaintextEncryptionManager(), Collections.emptyList()); return new IcebergSourceReader<>( - RecordEmitters.emitter(), readerMetrics, readerFunction, splitComparator, readerContext); + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); } private static class IdBasedComparator implements SerializableComparator { From 7283b56581eb7431a673fc05df4a42c70566da57 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 27 Sep 2023 21:32:50 +0200 Subject: [PATCH 06/15] Sundar's comments --- .../java/org/apache/iceberg/flink/source/IcebergSource.java | 2 +- .../iceberg/flink/source/reader/SerializableRecordEmitter.java | 1 + .../org/apache/iceberg/flink/source/split/SplitComparators.java | 2 +- .../flink/source/TestIcebergSourceWithWatermarkExtractor.java | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 5f2aa068a3f7..e97f33a3683e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -515,7 +515,7 @@ public IcebergSource build() { emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); splitAssignerFactory = new OrderedSplitAssignerFactory( - SplitComparators.watermarkComparator(watermarkExtractor)); + SplitComparators.watermarksAwareComparator(watermarkExtractor)); } checkRequired(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index c8c8adc0a440..f1a6679315e3 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -22,6 +22,7 @@ import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +@FunctionalInterface public interface SerializableRecordEmitter extends RecordEmitter, T, IcebergSourceSplit>, Serializable { static SerializableRecordEmitter defaultEmitter() { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index cda7437e2413..b975e5116d0e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -59,7 +59,7 @@ public static SerializableComparator fileSequenceNumber() { } /** Comparator which orders the splits based on watermark of the splits */ - public static SerializableComparator watermarkComparator( + public static SerializableComparator watermarksAwareComparator( IcebergWatermarkExtractor watermarkExtractor) { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { long watermark1 = watermarkExtractor.extractWatermark(o1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8844ae4e2a1a..b14f832b034d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -224,7 +224,7 @@ public void testThrottling() throws Exception { // (100 min - 20 min - 0 min) // Also this validates that the WatermarkAlignment is working Awaitility.await() - .atMost(5, TimeUnit.SECONDS) + .atMost(10, TimeUnit.SECONDS) .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); From 4f11bbaa995234f91c58aaf0026991412742ed85 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 28 Sep 2023 10:31:48 +0200 Subject: [PATCH 07/15] Small fixes --- .../iceberg/flink/source/reader/SerializableRecordEmitter.java | 2 ++ .../flink/source/reader/WatermarkExtractorRecordEmitter.java | 2 +- .../source/TestIcebergSourceFailoverWithWatermarkExtractor.java | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index f1a6679315e3..1a2010e51895 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -19,9 +19,11 @@ package org.apache.iceberg.flink.source.reader; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +@Internal @FunctionalInterface public interface SerializableRecordEmitter extends RecordEmitter, T, IcebergSourceSplit>, Serializable { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index 2b6e38f8ad39..1f6b2a02e75c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -46,7 +46,7 @@ public void emitRecord( if (!split.splitId().equals(lastSplit)) { long newWatermark = timeExtractor.extractWatermark(split); if (newWatermark < watermark) { - LOG.warn( + LOG.info( "Watermark decreased. PreviousWM {}, currentWM {}, previousSplit {}, currentSplit {}.", watermark, newWatermark, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index 50bcacbe2b1d..5d2317d3e823 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -41,7 +41,7 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); - // Within a batch, increment ts by 1 minute + // Within a batch, increment ts by 1 second private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); From 0734faf2d545721eb26d3293034971530cc97fcd Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 15 Nov 2023 12:57:59 +0100 Subject: [PATCH 08/15] Column name based split orders --- .../iceberg/flink/source/IcebergSource.java | 43 +++++++++--------- .../reader/IcebergWatermarkExtractor.java | 2 +- .../reader/SerializableRecordEmitter.java | 2 +- ... => TimestampBasedWatermarkExtractor.java} | 6 +-- .../WatermarkExtractorRecordEmitter.java | 1 + .../flink/source/split/SplitComparators.java | 2 +- .../source/TestIcebergSourceFailover.java | 12 +++-- ...gSourceFailoverWithWatermarkExtractor.java | 44 +++++++++---------- ...stIcebergSourceWithWatermarkExtractor.java | 14 +++--- 9 files changed, 63 insertions(+), 63 deletions(-) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergTimestampWatermarkExtractor.java => TimestampBasedWatermarkExtractor.java} (90%) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index e97f33a3683e..a9a511120615 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -65,11 +65,13 @@ import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.TimestampBasedWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,7 +225,7 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; - private IcebergWatermarkExtractor watermarkExtractor; + private String watermarkColumn; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -246,8 +248,8 @@ public Builder table(Table newTable) { public Builder assignerFactory(SplitAssignerFactory assignerFactory) { Preconditions.checkArgument( - watermarkExtractor == null, - "WatermarkExtractor and SplitAssigner should not be set in the same source"); + watermarkColumn == null, + "Watermark column and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -441,16 +443,14 @@ public Builder setAll(Map properties) { } /** - * Sets the {@link IcebergWatermarkExtractor} to retrieve the split watermark before emitting - * the records for a given split. The {@link - * IcebergWatermarkExtractor#extractWatermark(IcebergSourceSplit)} is also used for ordering the - * splits for read. + * Emits watermarks once per split based on the file statistics for the given split. The + * watermarks generated this way are also used for ordering the splits for read. */ - public Builder watermarkExtractor(IcebergWatermarkExtractor newWatermarkExtractor) { + public Builder watermarkColumn(String columnName) { Preconditions.checkArgument( splitAssignerFactory == null, - "WatermarkExtractor and SplitAssigner should not be set in the same source"); - this.watermarkExtractor = newWatermarkExtractor; + "Watermark column and SplitAssigner should not be set in the same source"); + this.watermarkColumn = columnName; return this; } @@ -478,6 +478,19 @@ public IcebergSource build() { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); } + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + IcebergWatermarkExtractor watermarkExtractor = + new TimestampBasedWatermarkExtractor(icebergSchema, watermarkColumn); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory( + SplitComparators.watermarksAwareComparator(watermarkExtractor)); + } + ScanContext context = contextBuilder.build(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { @@ -508,16 +521,6 @@ public IcebergSource build() { } } - SerializableRecordEmitter emitter; - if (watermarkExtractor == null) { - emitter = SerializableRecordEmitter.defaultEmitter(); - } else { - emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); - splitAssignerFactory = - new OrderedSplitAssignerFactory( - SplitComparators.watermarksAwareComparator(watermarkExtractor)); - } - checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource<>( diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java index 0221e42b6e9e..1d28eb6d938b 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java @@ -22,7 +22,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; /** The interface used to extract watermarks from splits. */ -public interface IcebergWatermarkExtractor extends Serializable { +public interface IcebergWatermarkExtractor extends Serializable { /** Get the watermark for a split. */ long extractWatermark(IcebergSourceSplit split); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index 1a2010e51895..20ddb47e4fed 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -36,6 +36,6 @@ static SerializableRecordEmitter defaultEmitter() { static SerializableRecordEmitter emitterWithWatermark( IcebergWatermarkExtractor extractor) { - return new WatermarkExtractorRecordEmitter(extractor); + return new WatermarkExtractorRecordEmitter<>(extractor); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java similarity index 90% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java index a2039b28e857..c95ff213670e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergTimestampWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.Comparator; -import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -33,8 +32,7 @@ * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by * the {@link WatermarkExtractorRecordEmitter} along with the actual records. */ -public class IcebergTimestampWatermarkExtractor - implements IcebergWatermarkExtractor, Serializable { +public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtractor, Serializable { private final int tsFieldId; /** @@ -43,7 +41,7 @@ public class IcebergTimestampWatermarkExtractor * @param schema The schema of the Table * @param tsFieldName The timestamp column which should be used as an event time */ - public IcebergTimestampWatermarkExtractor(Schema schema, String tsFieldName) { + 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"); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index 1f6b2a02e75c..e2101c16f9a1 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -53,6 +53,7 @@ public void emitRecord( lastSplit, split.splitId()); } + watermark = newWatermark; output.emitWatermark(new Watermark(watermark)); lastSplit = split.splitId(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index b975e5116d0e..6be7c86deef3 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -60,7 +60,7 @@ public static SerializableComparator fileSequenceNumber() { /** Comparator which orders the splits based on watermark of the splits */ public static SerializableComparator watermarksAwareComparator( - IcebergWatermarkExtractor watermarkExtractor) { + IcebergWatermarkExtractor watermarkExtractor) { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { long watermark1 = watermarkExtractor.extractWatermark(o1); long watermark2 = watermarkExtractor.extractWatermark(o2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 70e7a79d8373..7d991ee603c9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -39,6 +39,7 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -97,6 +98,11 @@ protected List generateRecords(int numRecords, long seed) { return RandomGenericData.generate(schema(), numRecords, seed); } + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -150,8 +156,7 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,8 +219,7 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index 5d2317d3e823..d99348b9852c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -34,9 +34,9 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.reader.IcebergTimestampWatermarkExtractor; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -50,9 +50,8 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg protected IcebergSource.Builder sourceBuilder() { return IcebergSource.builder() .tableLoader(sourceTableResource.tableLoader()) - .watermarkExtractor(new IcebergTimestampWatermarkExtractor(TestFixtures.TS_SCHEMA, "ts")) - .project(TestFixtures.TS_SCHEMA) - .includeColumnStats(true); + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); } @Override @@ -65,14 +64,13 @@ protected List generateRecords(int numRecords, long seed) { // Override the ts field to create a more realistic situation for event time alignment tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); return RandomGenericData.generate(schema(), numRecords, seed).stream() - .map( + .peek( record -> { LocalDateTime ts = LocalDateTime.ofInstant( Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), ZoneId.of("Z")); record.setField("ts", ts); - return record; }) .collect(Collectors.toList()); } @@ -84,32 +82,30 @@ record -> { * {@link LocalDateTime} to a Long type so that Comparators can continue to work. */ @Override - protected void assertRecords( - Table table, List expectedRecords, Duration interval, int maxCount) throws Exception { + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { List expectedNormalized = convertTimestampField(expectedRecords); - for (int i = 0; i < maxCount; ++i) { - if (SimpleDataUtil.equalsRecords( - expectedNormalized, - convertTimestampField(SimpleDataUtil.tableRecords(table)), - table.schema())) { - break; - } else { - Thread.sleep(interval.toMillis()); - } - } - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertTimestampField(SimpleDataUtil.tableRecords(table)), - table.schema()); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); } private List convertTimestampField(List records) { return records.stream() - .map( + .peek( r -> { LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); - return r; }) .collect(Collectors.toList()); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index b14f832b034d..090437eb2a85 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -66,7 +66,6 @@ import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.RowDataConverter; import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.reader.IcebergTimestampWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -155,7 +154,7 @@ public void testWindowing() throws Exception { public void apply( TimeWindow window, Iterable values, Collector out) { // Just print all the data to confirm everything has arrived - values.forEach(r -> out.collect(r)); + values.forEach(out::collect); } }); @@ -224,7 +223,7 @@ public void testThrottling() throws Exception { // (100 min - 20 min - 0 min) // Also this validates that the WatermarkAlignment is working Awaitility.await() - .atMost(10, TimeUnit.SECONDS) + .atMost(120, TimeUnit.SECONDS) .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); @@ -250,7 +249,7 @@ public void testThrottling() throws Exception { // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 // min - 15 min) - Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); + Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); // Add some new records which should unblock the throttled reader batch = ImmutableList.of(generateRecord(110, "File-5-110"), generateRecord(111, "File-5-111")); @@ -259,15 +258,14 @@ public void testThrottling() throws Exception { waitForRecords(resultIterator, 6); // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); + Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); } protected IcebergSource.Builder sourceBuilder() { return IcebergSource.builder() .tableLoader(sourceTableResource.tableLoader()) - .watermarkExtractor(new IcebergTimestampWatermarkExtractor(TestFixtures.TS_SCHEMA, "ts")) + .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) - .includeColumnStats(true) .splitSize(100L); } @@ -317,7 +315,7 @@ protected void assertRecords( Assert.assertEquals(expected, received); } - protected void waitForRecords(CollectResultIterator iterator, int num) throws Exception { + protected void waitForRecords(CollectResultIterator iterator, int num) { assertThat( CompletableFuture.supplyAsync( () -> { From b3c41bf28cc6fde8b87299743a65f1519640b9d2 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 16 Nov 2023 11:47:39 +0100 Subject: [PATCH 09/15] Steven's comments --- .../iceberg/flink/source/IcebergSource.java | 10 +- ...ava => ColumnStatsWatermarkExtractor.java} | 8 +- .../reader/SerializableRecordEmitter.java | 2 +- ...ctor.java => SplitWatermarkExtractor.java} | 2 +- .../WatermarkExtractorRecordEmitter.java | 17 +- .../flink/source/split/SplitComparators.java | 6 +- ...stIcebergSourceWithWatermarkExtractor.java | 184 ++++++++---------- 7 files changed, 100 insertions(+), 129 deletions(-) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{TimestampBasedWatermarkExtractor.java => ColumnStatsWatermarkExtractor.java} (87%) rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergWatermarkExtractor.java => SplitWatermarkExtractor.java} (94%) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index a9a511120615..e4a04bd07e23 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -60,12 +60,12 @@ import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.IcebergWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; -import org.apache.iceberg.flink.source.reader.TimestampBasedWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; @@ -483,12 +483,12 @@ public IcebergSource build() { // Column statistics is needed for watermark generation contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); - IcebergWatermarkExtractor watermarkExtractor = - new TimestampBasedWatermarkExtractor(icebergSchema, watermarkColumn); + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn); emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); splitAssignerFactory = new OrderedSplitAssignerFactory( - SplitComparators.watermarksAwareComparator(watermarkExtractor)); + SplitComparators.watermark(watermarkExtractor)); } ScanContext context = contextBuilder.build(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 87% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index c95ff213670e..ae08ef8ea800 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Comparator; +import org.apache.flink.annotation.Internal; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -28,11 +29,12 @@ import org.apache.iceberg.types.Types; /** - * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * {@link SplitWatermarkExtractor} 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 { +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { private final int tsFieldId; /** @@ -41,7 +43,7 @@ public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtract * @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) { + public ColumnStatsWatermarkExtractor(Schema schema, String tsFieldName) { Types.NestedField field = schema.findField(tsFieldName); Preconditions.checkArgument( field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index 20ddb47e4fed..d5edac579a34 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -35,7 +35,7 @@ static SerializableRecordEmitter defaultEmitter() { } static SerializableRecordEmitter emitterWithWatermark( - IcebergWatermarkExtractor extractor) { + SplitWatermarkExtractor extractor) { return new WatermarkExtractorRecordEmitter<>(extractor); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 94% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java index 1d28eb6d938b..d1c50ac8ca52 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java @@ -22,7 +22,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; /** The interface used to extract watermarks from splits. */ -public interface IcebergWatermarkExtractor extends Serializable { +public interface SplitWatermarkExtractor extends Serializable { /** Get the watermark for a split. */ long extractWatermark(IcebergSourceSplit split); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index e2101c16f9a1..023b35925e1e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -28,35 +28,36 @@ * Emitter which emits the watermarks, records and updates the split position. * *

The Emitter emits watermarks at the beginning of every split provided by the {@link - * IcebergWatermarkExtractor}. + * SplitWatermarkExtractor}. */ class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); - private final IcebergWatermarkExtractor timeExtractor; - private String lastSplit = null; + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; private long watermark; - WatermarkExtractorRecordEmitter(IcebergWatermarkExtractor timeExtractor) { + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { this.timeExtractor = timeExtractor; } @Override public void emitRecord( RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - if (!split.splitId().equals(lastSplit)) { + if (!split.splitId().equals(lastSplitId)) { long newWatermark = timeExtractor.extractWatermark(split); if (newWatermark < watermark) { LOG.info( - "Watermark decreased. PreviousWM {}, currentWM {}, previousSplit {}, currentSplit {}.", + "previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", watermark, newWatermark, - lastSplit, + lastSplitId, split.splitId()); } watermark = newWatermark; output.emitWatermark(new Watermark(watermark)); - lastSplit = split.splitId(); + lastSplitId = split.splitId(); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); } output.collect(element.record()); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 6be7c86deef3..56ee92014d12 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source.split; -import org.apache.iceberg.flink.source.reader.IcebergWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -59,8 +59,8 @@ public static SerializableComparator fileSequenceNumber() { } /** Comparator which orders the splits based on watermark of the splits */ - public static SerializableComparator watermarksAwareComparator( - IcebergWatermarkExtractor watermarkExtractor) { + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { long watermark1 = watermarkExtractor.extractWatermark(o1); long watermark2 = watermarkExtractor.extractWatermark(o2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 090437eb2a85..ddb9b5235047 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -37,7 +36,6 @@ import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; @@ -48,20 +46,16 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; -import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.RowData; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.RowDataConverter; @@ -69,8 +63,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; import org.junit.Assert; import org.junit.ClassRule; @@ -109,25 +101,24 @@ public void testWindowing() throws Exception { // Generate records with the following pattern: // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "File-1-100"), (103, "File-1-103") + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "File-2-0"), (1, "File-2-1"),... - // - Split 2 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... // - File 3 - Parallel write for the first records (Watermark 60000) - // - Split 1 - 2 records (1, "File-3-1"), (3, "File-3-3") - List batch; - batch = ImmutableList.of(generateRecord(100, "100"), generateRecord(103, "103")); + // - Split 1 - 2 records (1, "file_3-recordTs_1"), (3, "file_3-recordTs_3") + List batch = ImmutableList.of(generateRecord(100, "100"), generateRecord(103, "103")); expectedRecords.addAll(batch); dataAppender.appendToTable(batch); batch = Lists.newArrayListWithCapacity(100); for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch.add(generateRecord(i % 5, "File-2-" + i)); + batch.add(generateRecord(i % 5, "file_2-recordTs_" + i)); } expectedRecords.addAll(batch); dataAppender.appendToTable(batch); - batch = ImmutableList.of(generateRecord(1, "File-3-1"), generateRecord(3, "File-3-3")); + batch = ImmutableList.of(generateRecord(1, "file_3-recordTs_1"), generateRecord(3, "file_3-recordTs_3")); expectedRecords.addAll(batch); dataAppender.appendToTable(batch); @@ -158,18 +149,16 @@ public void apply( } }); - CollectResultIterator resultIterator = addCollectSink(windowed); - - // Start the job - JobClient jobClient = env.executeAsync("Iceberg Source Windowing Test"); - resultIterator.setJobClient(jobClient); + try (CloseableIterator resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); - // Write data so the windows containing test data are closed - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + // Write data so the windows containing test data are closed + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); - assertRecords(resultIterator, expectedRecords); + assertRecords(resultIterator, expectedRecords); + } } @Test @@ -178,17 +167,17 @@ public void testThrottling() throws Exception { // Generate records with the following pattern: // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "File-1-100"), (103, "File-1-103") + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "File-2-0"), (1, "File-2-1"),... - // - Split 2 - 100 records (0, "File-2-0"), (1, "File-2-1"),... + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... List batch; - batch = ImmutableList.of(generateRecord(100, "File-1-100"), generateRecord(103, "File-1-103")); + batch = ImmutableList.of(generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); dataAppender.appendToTable(batch); batch = Lists.newArrayListWithCapacity(100); for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch.add(generateRecord(i % 5, "File-2-" + i)); + batch.add(generateRecord(i % 5, "file_2-recordTs_" + i)); } dataAppender.appendToTable(batch); @@ -208,57 +197,55 @@ public void testThrottling() throws Exception { SOURCE_NAME, TypeInformation.of(RowData.class)); - CollectResultIterator resultIterator = addCollectSink(stream); - - // Start the job - JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); - resultIterator.setJobClient(jobClient); - - // Check that the read the non-blocked data - // The first RECORD_NUM_FOR_2_SPLITS should be read - // 1 or more from the runaway reader should be arrived depending on thread scheduling - waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); - - // Get the drift metric, wait for it to be created and reach the expected state - // (100 min - 20 min - 0 min) - // Also this validates that the WatermarkAlignment is working - Awaitility.await() - .atMost(120, TimeUnit.SECONDS) - .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); - Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); - - // Add some old records with 2 splits, so even if the blocked gets one split, the other reader - // one gets one as well - batch = - ImmutableList.of( - generateRecord(15, "File-3-15"), - generateRecord(16, "File-3-16"), - generateRecord(17, "File-3-17")); - dataAppender.appendToTable(batch); - batch = - ImmutableList.of( - generateRecord(15, "File-4-15"), - generateRecord(16, "File-4-16"), - generateRecord(17, "File-4-17")); - dataAppender.appendToTable(batch); - // The records received will highly depend on scheduling - // We minimally get 3 records from the non-blocked reader - // We might get 1 record from the blocked reader (as part of the previous batch - File-1) - // We might get 3 records form the non-blocked reader if it gets both new splits - waitForRecords(resultIterator, 3); - - // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 - // min - 15 min) - Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); - - // Add some new records which should unblock the throttled reader - batch = ImmutableList.of(generateRecord(110, "File-5-110"), generateRecord(111, "File-5-111")); - dataAppender.appendToTable(batch); - // We should get all the records at this point - waitForRecords(resultIterator, 6); - - // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); + Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + batch = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + dataAppender.appendToTable(batch); + batch = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable(batch); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - file1-record-ts) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); + + // Add some new records which should unblock the throttled reader + batch = ImmutableList.of(generateRecord(110, "file_5-recordTs_110"), generateRecord(111, "file_5-recordTs_111")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); + } } protected IcebergSource.Builder sourceBuilder() { @@ -281,14 +268,8 @@ protected Record generateRecord(int minutes, String str) { return record; } - /** - * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves - * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates - * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the - * {@link LocalDateTime} to a Long type so that Comparators can continue to work. - */ protected void assertRecords( - CollectResultIterator iterator, List expectedRecords) throws Exception { + CloseableIterator iterator, List expectedRecords) throws Exception { Set received = Sets.newHashSetWithExpectedSize(expectedRecords.size()); @@ -300,10 +281,12 @@ protected void assertRecords( received.add(iterator.next()); count++; } + if (count < expectedRecords.size()) { throw new IllegalStateException( String.format("Fail to get %d records.", expectedRecords.size())); } + return true; })) .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); @@ -315,7 +298,7 @@ protected void assertRecords( Assert.assertEquals(expected, received); } - protected void waitForRecords(CollectResultIterator iterator, int num) { + protected void waitForRecords(CloseableIterator iterator, int num) { assertThat( CompletableFuture.supplyAsync( () -> { @@ -324,31 +307,16 @@ protected void waitForRecords(CollectResultIterator iterator, int num) iterator.next(); count++; } + if (count < num) { throw new IllegalStateException(String.format("Fail to get %d records.", num)); } + return true; })) .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); } - private CollectResultIterator addCollectSink(DataStream stream) { - TypeSerializer serializer = - stream.getType().createSerializer(stream.getExecutionConfig()); - String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); - CollectSinkOperatorFactory factory = - new CollectSinkOperatorFactory<>(serializer, accumulatorName); - CollectSinkOperator operator = (CollectSinkOperator) factory.getOperator(); - CollectStreamSink sink = new CollectStreamSink<>(stream, factory); - sink.name("Data stream collect sink"); - stream.getExecutionEnvironment().addOperator(sink.getTransformation()); - return new CollectResultIterator<>( - operator.getOperatorIdFuture(), - serializer, - accumulatorName, - stream.getExecutionEnvironment().getCheckpointConfig()); - } - private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; return reporter.findMetrics(jobID, metricsName).values().stream() From 6f1f5211496305acfb5d3037fc7c71b8b153941b Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 16 Nov 2023 16:07:37 +0100 Subject: [PATCH 10/15] Add long type as a watermark column --- .../iceberg/flink/source/IcebergSource.java | 25 +- .../reader/ColumnStatsWatermarkExtractor.java | 29 ++- .../reader/SerializableRecordEmitter.java | 3 +- ...stIcebergSourceWithWatermarkExtractor.java | 20 +- .../TestColumnStatsWatermarkExtractor.java | 215 ++++++++++++++++++ 5 files changed, 267 insertions(+), 25 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index e4a04bd07e23..cf745a35c345 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.Boundedness; @@ -58,14 +59,14 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; @@ -226,6 +227,7 @@ public static class Builder { private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private String watermarkColumn; + private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -444,7 +446,9 @@ public Builder setAll(Map properties) { /** * Emits watermarks once per split based on the file statistics for the given split. The - * watermarks generated this way are also used for ordering the splits for read. + * generated watermarks are also used for ordering the splits for read. Accepted column types + * are timestamp/timestamptz/long. For long columns consider setting {@link + * #watermarkTimeUnit(TimeUnit)}. */ public Builder watermarkColumn(String columnName) { Preconditions.checkArgument( @@ -454,6 +458,16 @@ public Builder watermarkColumn(String columnName) { return this; } + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkTimeUnit(TimeUnit timeUnit) { + this.watermarkTimeUnit = timeUnit; + return this; + } + /** @deprecated Use {@link #setAll} instead. */ @Deprecated public Builder properties(Map properties) { @@ -484,11 +498,10 @@ public IcebergSource build() { contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); SplitWatermarkExtractor watermarkExtractor = - new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn); + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); splitAssignerFactory = - new OrderedSplitAssignerFactory( - SplitComparators.watermark(watermarkExtractor)); + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); } ScanContext context = contextBuilder.build(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index ae08ef8ea800..39beb5c1e5f5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -20,34 +20,42 @@ import java.io.Serializable; import java.util.Comparator; +import java.util.concurrent.TimeUnit; import org.apache.flink.annotation.Internal; 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.Type.TypeID; import org.apache.iceberg.types.Types; /** - * {@link SplitWatermarkExtractor} 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. + * {@link SplitWatermarkExtractor} 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. */ @Internal public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { private final int tsFieldId; + private final TimeUnit timeUnit; /** * Creates the extractor. * * @param schema The schema of the Table - * @param tsFieldName The timestamp column which should be used as an event time + * @param tsFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds */ - public ColumnStatsWatermarkExtractor(Schema schema, String tsFieldName) { + public ColumnStatsWatermarkExtractor(Schema schema, String tsFieldName, TimeUnit timeUnit) { Types.NestedField field = schema.findField(tsFieldName); + TypeID typeID = field.type().typeId(); Preconditions.checkArgument( - field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); this.tsFieldId = field.fieldId(); + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; } @Override @@ -55,10 +63,9 @@ public long extractWatermark(IcebergSourceSplit split) { return split.task().files().stream() .map( scanTask -> - (long) - Conversions.fromByteBuffer( - Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)) - / 1000L) + timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)))) .min(Comparator.comparingLong(l -> l)) .get(); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java index d5edac579a34..a6e2c1dae243 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -34,8 +34,7 @@ static SerializableRecordEmitter defaultEmitter() { }; } - static SerializableRecordEmitter emitterWithWatermark( - SplitWatermarkExtractor extractor) { + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { return new WatermarkExtractorRecordEmitter<>(extractor); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index ddb9b5235047..6b9eb1baf833 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -118,7 +118,9 @@ public void testWindowing() throws Exception { expectedRecords.addAll(batch); dataAppender.appendToTable(batch); - batch = ImmutableList.of(generateRecord(1, "file_3-recordTs_1"), generateRecord(3, "file_3-recordTs_3")); + batch = + ImmutableList.of( + generateRecord(1, "file_3-recordTs_1"), generateRecord(3, "file_3-recordTs_3")); expectedRecords.addAll(batch); dataAppender.appendToTable(batch); @@ -172,7 +174,9 @@ public void testThrottling() throws Exception { // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... List batch; - batch = ImmutableList.of(generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); dataAppender.appendToTable(batch); batch = Lists.newArrayListWithCapacity(100); @@ -229,7 +233,8 @@ public void testThrottling() throws Exception { dataAppender.appendToTable(batch); // The records received will highly depend on scheduling // We minimally get 3 records from the non-blocked reader - // We might get 1 record from the blocked reader (as part of the previous batch - file1-record-ts) + // We might get 1 record from the blocked reader (as part of the previous batch - + // file1-record-ts) // We might get 3 records form the non-blocked reader if it gets both new splits waitForRecords(resultIterator, 3); @@ -238,7 +243,10 @@ public void testThrottling() throws Exception { Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); // Add some new records which should unblock the throttled reader - batch = ImmutableList.of(generateRecord(110, "file_5-recordTs_110"), generateRecord(111, "file_5-recordTs_111")); + batch = + ImmutableList.of( + generateRecord(110, "file_5-recordTs_110"), + generateRecord(111, "file_5-recordTs_111")); dataAppender.appendToTable(batch); // We should get all the records at this point waitForRecords(resultIterator, 6); @@ -268,8 +276,8 @@ protected Record generateRecord(int minutes, String str) { return record; } - protected void assertRecords( - CloseableIterator iterator, List expectedRecords) throws Exception { + protected void assertRecords(CloseableIterator iterator, List expectedRecords) + throws Exception { Set received = Sets.newHashSetWithExpectedSize(expectedRecords.size()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..50d04c7240f2 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,215 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "ts", Types.TimestampType.withoutZone()), + required(2, "tstz", Types.TimestampType.withZone()), + required(3, "l", Types.LongType.get()), + required(4, "s", Types.StringType.get())); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private GenericAppenderHelper dataAppender; + private long minTs = Long.MAX_VALUE; + private long minTsTz = Long.MAX_VALUE; + private long minL = Long.MAX_VALUE; + + @Before + public void initTable() throws IOException { + dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + List batch = RandomGenericData.generate(SCHEMA, 3, 2L); + dataAppender.appendToTable(batch); + + for (Record r : batch) { + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minTs = Math.min(minTs, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minTsTz = Math.min(minTsTz, offsetDateTime.toInstant().toEpochMilli()); + + minL = Math.min(minL, (Long) r.get(2)); + } + } + + @Test + public void testTimestamp() { + DataFile dataFile = + sourceTableResource + .table() + .currentSnapshot() + .addedDataFiles(sourceTableResource.table().io()) + .iterator() + .next(); + ColumnStatsWatermarkExtractor tsExtractor = + new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); + + Assert.assertEquals( + minTs, + tsExtractor.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + } + + @Test + public void testTimestampWithTz() { + DataFile dataFile = + sourceTableResource + .table() + .currentSnapshot() + .addedDataFiles(sourceTableResource.table().io()) + .iterator() + .next(); + ColumnStatsWatermarkExtractor tsTzExtractor = + new ColumnStatsWatermarkExtractor(SCHEMA, "tstz", null); + + Assert.assertEquals( + minTsTz, + tsTzExtractor.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + } + + @Test + public void testLong() { + DataFile dataFile = + sourceTableResource + .table() + .currentSnapshot() + .addedDataFiles(sourceTableResource.table().io()) + .iterator() + .next(); + ColumnStatsWatermarkExtractor longExtractorMilliSeconds = + new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MILLISECONDS); + ColumnStatsWatermarkExtractor longExtractorMicroSeconds = + new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MICROSECONDS); + + Assert.assertEquals( + minL, + longExtractorMilliSeconds.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + Assert.assertEquals( + minL / 1000L, + longExtractorMicroSeconds.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + } + + @Test + public void testMultipleFiles() throws IOException { + DataFile oldDataFile = + sourceTableResource + .table() + .currentSnapshot() + .addedDataFiles(sourceTableResource.table().io()) + .iterator() + .next(); + List batch = RandomGenericData.generate(SCHEMA, 3, 19L); + dataAppender.appendToTable(batch); + + long minTsNew = Long.MAX_VALUE; + for (Record r : batch) { + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minTsNew = Math.min(minTsNew, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); + } + + DataFile newDataFile = + sourceTableResource + .table() + .currentSnapshot() + .addedDataFiles(sourceTableResource.table().io()) + .iterator() + .next(); + ColumnStatsWatermarkExtractor tsExtractor = + new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); + + Assert.assertEquals( + minTsNew, + tsExtractor.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile)))); + Assert.assertEquals( + minTs, + tsExtractor.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(oldDataFile)))); + Assert.assertEquals( + Math.min(minTsNew, minTs), + tsExtractor.extractWatermark( + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile, oldDataFile)))); + } + + @Test + public void testWrongColumn() { + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, "s", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + private static class DummyTask extends BaseCombinedScanTask { + private Collection files; + + DummyTask(DataFile... dataFiles) { + files = + Arrays.stream(dataFiles).map(f -> new MockFileScanTask(f)).collect(Collectors.toList()); + } + + @Override + public Collection files() { + return files; + } + } +} From 6e2cda8087ca75fd49949183e147ec8e66a26d4f Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Fri, 17 Nov 2023 12:01:55 +0100 Subject: [PATCH 11/15] Steven's stuff --- .../reader/ColumnStatsWatermarkExtractor.java | 32 ++++++--- .../WatermarkExtractorRecordEmitter.java | 7 +- ...stIcebergSourceWithWatermarkExtractor.java | 18 +++-- .../TestColumnStatsWatermarkExtractor.java | 69 ++++++++----------- 4 files changed, 67 insertions(+), 59 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index 39beb5c1e5f5..9d3feec46c23 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -36,36 +36,50 @@ */ @Internal public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { - private final int tsFieldId; + private final int eventTimeFieldId; private final TimeUnit timeUnit; /** * Creates the extractor. * * @param schema The schema of the Table - * @param tsFieldName The column which should be used as an event time + * @param eventTimeFieldId The column which should be used as an event time * @param timeUnit Used for converting the long value to epoch milliseconds */ - public ColumnStatsWatermarkExtractor(Schema schema, String tsFieldName, TimeUnit timeUnit) { - Types.NestedField field = schema.findField(tsFieldName); + public ColumnStatsWatermarkExtractor(Schema schema, String eventTimeFieldId, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldId); TypeID typeID = field.type().typeId(); Preconditions.checkArgument( typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", typeID); - this.tsFieldId = field.fieldId(); + this.eventTimeFieldId = field.fieldId(); // Use the timeUnit only for Long columns. this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; } + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws NullPointerException if there is no statistics for the column + */ @Override public long extractWatermark(IcebergSourceSplit split) { return split.task().files().stream() .map( - scanTask -> - timeUnit.toMillis( - Conversions.fromByteBuffer( - Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)))) + scanTask -> { + Preconditions.checkNotNull( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics in file = %s for columnId = %s", + scanTask.file(), + (Object) Integer.valueOf(eventTimeFieldId)); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) .min(Comparator.comparingLong(l -> l)) .get(); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index 023b35925e1e..9ae18c4cff7c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -52,12 +52,13 @@ public void emitRecord( newWatermark, lastSplitId, split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); } - watermark = newWatermark; - output.emitWatermark(new Watermark(watermark)); lastSplitId = split.splitId(); - LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); } output.collect(element.record()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 6b9eb1baf833..badae552723f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -197,7 +197,7 @@ public void testThrottling() throws Exception { .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(), WatermarkStrategy.noWatermarks() - .withWatermarkAlignment("iceberg", Duration.ofMinutes(20)), + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), SOURCE_NAME, TypeInformation.of(RowData.class)); @@ -214,8 +214,12 @@ public void testThrottling() throws Exception { // Also this validates that the WatermarkAlignment is working Awaitility.await() .atMost(120, TimeUnit.SECONDS) - .until(() -> findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).isPresent()); - Gauge drift = findAlignmentDriftMetric(jobClient.getJobID(), 4800000L).get(); + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); // Add some old records with 2 splits, so even if the blocked gets one split, the other reader // one gets one as well @@ -240,7 +244,9 @@ public void testThrottling() throws Exception { // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 // min - 15 min) - Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() == 3900000L); + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); // Add some new records which should unblock the throttled reader batch = @@ -252,7 +258,9 @@ public void testThrottling() throws Exception { waitForRecords(resultIterator, 6); // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> drift.getValue() < 1200000L); + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 50d04c7240f2..3334461dc920 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -65,9 +65,10 @@ public class TestColumnStatsWatermarkExtractor { new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); private GenericAppenderHelper dataAppender; - private long minTs = Long.MAX_VALUE; - private long minTsTz = Long.MAX_VALUE; - private long minL = Long.MAX_VALUE; + private long timestampFieldMinValue = Long.MAX_VALUE; + private long timestampTzFieldMinValue = Long.MAX_VALUE; + private long longFieldMinValue = Long.MAX_VALUE; + private DataFile dataFile; @Before public void initTable() throws IOException { @@ -80,91 +81,75 @@ public void initTable() throws IOException { for (Record r : batch) { LocalDateTime localDateTime = (LocalDateTime) r.get(0); - minTs = Math.min(minTs, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); + timestampFieldMinValue = + Math.min(timestampFieldMinValue, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); - minTsTz = Math.min(minTsTz, offsetDateTime.toInstant().toEpochMilli()); + timestampTzFieldMinValue = + Math.min(timestampTzFieldMinValue, offsetDateTime.toInstant().toEpochMilli()); - minL = Math.min(minL, (Long) r.get(2)); + longFieldMinValue = Math.min(longFieldMinValue, (Long) r.get(2)); } - } - @Test - public void testTimestamp() { - DataFile dataFile = + dataFile = sourceTableResource .table() .currentSnapshot() .addedDataFiles(sourceTableResource.table().io()) .iterator() .next(); + } + + @Test + public void testTimestamp() { ColumnStatsWatermarkExtractor tsExtractor = new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); Assert.assertEquals( - minTs, + timestampFieldMinValue, tsExtractor.extractWatermark( IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); } @Test public void testTimestampWithTz() { - DataFile dataFile = - sourceTableResource - .table() - .currentSnapshot() - .addedDataFiles(sourceTableResource.table().io()) - .iterator() - .next(); ColumnStatsWatermarkExtractor tsTzExtractor = new ColumnStatsWatermarkExtractor(SCHEMA, "tstz", null); Assert.assertEquals( - minTsTz, + timestampTzFieldMinValue, tsTzExtractor.extractWatermark( IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); } @Test public void testLong() { - DataFile dataFile = - sourceTableResource - .table() - .currentSnapshot() - .addedDataFiles(sourceTableResource.table().io()) - .iterator() - .next(); ColumnStatsWatermarkExtractor longExtractorMilliSeconds = new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MILLISECONDS); ColumnStatsWatermarkExtractor longExtractorMicroSeconds = new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MICROSECONDS); Assert.assertEquals( - minL, + longFieldMinValue, longExtractorMilliSeconds.extractWatermark( IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); Assert.assertEquals( - minL / 1000L, + longFieldMinValue / 1000L, longExtractorMicroSeconds.extractWatermark( IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); } @Test public void testMultipleFiles() throws IOException { - DataFile oldDataFile = - sourceTableResource - .table() - .currentSnapshot() - .addedDataFiles(sourceTableResource.table().io()) - .iterator() - .next(); List batch = RandomGenericData.generate(SCHEMA, 3, 19L); dataAppender.appendToTable(batch); - long minTsNew = Long.MAX_VALUE; + long timestampFieldMinValueNew = Long.MAX_VALUE; for (Record r : batch) { LocalDateTime localDateTime = (LocalDateTime) r.get(0); - minTsNew = Math.min(minTsNew, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); + timestampFieldMinValueNew = + Math.min( + timestampFieldMinValueNew, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); } DataFile newDataFile = @@ -178,17 +163,17 @@ public void testMultipleFiles() throws IOException { new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); Assert.assertEquals( - minTsNew, + timestampFieldMinValueNew, tsExtractor.extractWatermark( IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile)))); Assert.assertEquals( - minTs, + timestampFieldMinValue, tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(oldDataFile)))); + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); Assert.assertEquals( - Math.min(minTsNew, minTs), + Math.min(timestampFieldMinValue, timestampFieldMinValueNew), tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile, oldDataFile)))); + IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile, dataFile)))); } @Test From f80ffb54043ae7fc309710f0f80213ef9abb6be1 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Fri, 17 Nov 2023 16:22:49 +0100 Subject: [PATCH 12/15] Using ReaderUtil for tests --- .../reader/ColumnStatsWatermarkExtractor.java | 11 +- .../flink/source/reader/ReaderUtil.java | 8 +- .../TestColumnStatsWatermarkExtractor.java | 189 ++++++++---------- 3 files changed, 98 insertions(+), 110 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index 9d3feec46c23..fb4a492798c1 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -24,6 +24,7 @@ import org.apache.flink.annotation.Internal; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type.TypeID; @@ -58,19 +59,25 @@ public ColumnStatsWatermarkExtractor(Schema schema, String eventTimeFieldId, Tim this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; } + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId) { + this.eventTimeFieldId = eventTimeFieldId; + this.timeUnit = TimeUnit.MICROSECONDS; + } + /** * Get the watermark for a split using column statistics. * * @param split The split * @return The watermark - * @throws NullPointerException if there is no statistics for the column + * @throws IllegalArgumentException if there is no statistics for the column */ @Override public long extractWatermark(IcebergSourceSplit split) { return split.task().files().stream() .map( scanTask -> { - Preconditions.checkNotNull( + Preconditions.checkArgument( scanTask.file().lowerBounds() != null && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, "Missing statistics in file = %s for columnId = %s", diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f9ceaf842263..2a2503ef2478 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -60,9 +60,12 @@ public static FileScanTask createFileTask( FileFormat fileFormat, FileAppenderFactory appenderFactory) throws IOException { - try (FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat)) { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { appender.addAll(records); + } finally { + appender.close(); } DataFile dataFile = @@ -71,6 +74,7 @@ public static FileScanTask createFileTask( .withFileSizeInBytes(file.length()) .withPath(file.toString()) .withFormat(fileFormat) + .withMetrics(appender.metrics()) .build(); ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 3334461dc920..154a89b56f59 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -24,39 +24,49 @@ import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; -import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MockFileScanTask; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; +import org.junit.Assume; +import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(Parameterized.class) public class TestColumnStatsWatermarkExtractor { public static final Schema SCHEMA = new Schema( - required(1, "ts", Types.TimestampType.withoutZone()), - required(2, "tstz", Types.TimestampType.withZone()), - required(3, "l", Types.LongType.get()), - required(4, "s", Types.StringType.get())); + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -64,137 +74,104 @@ public class TestColumnStatsWatermarkExtractor { public final HadoopTableResource sourceTableResource = new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); - private GenericAppenderHelper dataAppender; - private long timestampFieldMinValue = Long.MAX_VALUE; - private long timestampTzFieldMinValue = Long.MAX_VALUE; - private long longFieldMinValue = Long.MAX_VALUE; - private DataFile dataFile; - - @Before - public void initTable() throws IOException { - dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + private final String columnName; - List batch = RandomGenericData.generate(SCHEMA, 3, 2L); - dataAppender.appendToTable(batch); + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); - for (Record r : batch) { - LocalDateTime localDateTime = (LocalDateTime) r.get(0); - timestampFieldMinValue = - Math.min(timestampFieldMinValue, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); - OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); - timestampTzFieldMinValue = - Math.min(timestampTzFieldMinValue, offsetDateTime.toInstant().toEpochMilli()); + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); - longFieldMinValue = Math.min(longFieldMinValue, (Long) r.get(2)); + minValues.merge("long_column", (Long) r.get(2), Math::min); + } } - - dataFile = - sourceTableResource - .table() - .currentSnapshot() - .addedDataFiles(sourceTableResource.table().io()) - .iterator() - .next(); } - @Test - public void testTimestamp() { - ColumnStatsWatermarkExtractor tsExtractor = - new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } - Assert.assertEquals( - timestampFieldMinValue, - tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; } @Test - public void testTimestampWithTz() { - ColumnStatsWatermarkExtractor tsTzExtractor = - new ColumnStatsWatermarkExtractor(SCHEMA, "tstz", null); + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); Assert.assertEquals( - timestampTzFieldMinValue, - tsTzExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); } @Test - public void testLong() { - ColumnStatsWatermarkExtractor longExtractorMilliSeconds = - new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MILLISECONDS); - ColumnStatsWatermarkExtractor longExtractorMicroSeconds = - new ColumnStatsWatermarkExtractor(SCHEMA, "l", TimeUnit.MICROSECONDS); + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); Assert.assertEquals( - longFieldMinValue, - longExtractorMilliSeconds.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); - Assert.assertEquals( - longFieldMinValue / 1000L, - longExtractorMicroSeconds.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); } @Test public void testMultipleFiles() throws IOException { - List batch = RandomGenericData.generate(SCHEMA, 3, 19L); - dataAppender.appendToTable(batch); - - long timestampFieldMinValueNew = Long.MAX_VALUE; - for (Record r : batch) { - LocalDateTime localDateTime = (LocalDateTime) r.get(0); - timestampFieldMinValueNew = - Math.min( - timestampFieldMinValueNew, localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); - } + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); - DataFile newDataFile = - sourceTableResource - .table() - .currentSnapshot() - .addedDataFiles(sourceTableResource.table().io()) - .iterator() - .next(); - ColumnStatsWatermarkExtractor tsExtractor = - new ColumnStatsWatermarkExtractor(SCHEMA, "ts", null); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); Assert.assertEquals( - timestampFieldMinValueNew, - tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile)))); + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); Assert.assertEquals( - timestampFieldMinValue, - tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(dataFile)))); + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); Assert.assertEquals( - Math.min(timestampFieldMinValue, timestampFieldMinValueNew), - tsExtractor.extractWatermark( - IcebergSourceSplit.fromCombinedScanTask(new DummyTask(newDataFile, dataFile)))); + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); } @Test public void testWrongColumn() { - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, "s", null)) + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); } - private static class DummyTask extends BaseCombinedScanTask { - private Collection files; + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); - DummyTask(DataFile... dataFiles) { - files = - Arrays.stream(dataFiles).map(f -> new MockFileScanTask(f)).collect(Collectors.toList()); - } + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(10); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics in file"); + } - @Override - public Collection files() { - return files; - } + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); } } From 3ccdcf1e149beb57931bcafce9862f40a2a8c9bc Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Tue, 21 Nov 2023 09:31:51 +0100 Subject: [PATCH 13/15] Added TestWatermarkBasedSplitAssigner tests --- .../iceberg/data/GenericAppenderHelper.java | 5 + .../reader/ColumnStatsWatermarkExtractor.java | 19 +- .../WatermarkExtractorRecordEmitter.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 169 +++++++++++++----- .../assigner/SplitAssignerTestBase.java | 21 +-- ...tFileSequenceNumberBasedSplitAssigner.java | 10 +- .../TestWatermarkBasedSplitAssigner.java | 146 +++++++++++++++ .../TestColumnStatsWatermarkExtractor.java | 5 +- 8 files changed, 303 insertions(+), 74 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java index 03e220f10ed3..beccc8bb70bf 100644 --- a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java +++ b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java @@ -39,6 +39,7 @@ public class GenericAppenderHelper { private static final String ORC_CONFIG_PREFIX = "^orc.*"; + private static final String PARQUET_CONFIG_PATTERN = ".*parquet.*"; private final Table table; private final FileFormat fileFormat; @@ -120,6 +121,10 @@ private static DataFile appendToLocalFile( appenderFactory.setAll(conf.getValByRegex(ORC_CONFIG_PREFIX)); } + if (FileFormat.PARQUET.equals(format) && conf != null) { + appenderFactory.setAll(conf.getValByRegex(PARQUET_CONFIG_PATTERN)); + } + FileAppender appender = appenderFactory.newAppender(Files.localOutput(file), format); try (FileAppender fileAppender = appender) { fileAppender.addAll(records); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index fb4a492798c1..44a1f9c3e47e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -38,30 +38,34 @@ @Internal public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { private final int eventTimeFieldId; + private final String eventTimeFieldName; private final TimeUnit timeUnit; /** * Creates the extractor. * * @param schema The schema of the Table - * @param eventTimeFieldId The column which should be used as an event time + * @param eventTimeFieldName The column which should be used as an event time * @param timeUnit Used for converting the long value to epoch milliseconds */ - public ColumnStatsWatermarkExtractor(Schema schema, String eventTimeFieldId, TimeUnit timeUnit) { - Types.NestedField field = schema.findField(eventTimeFieldId); + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); TypeID typeID = field.type().typeId(); Preconditions.checkArgument( typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", typeID); this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; // Use the timeUnit only for Long columns. this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; } @VisibleForTesting - ColumnStatsWatermarkExtractor(int eventTimeFieldId) { + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; this.timeUnit = TimeUnit.MICROSECONDS; } @@ -80,9 +84,10 @@ public long extractWatermark(IcebergSourceSplit split) { Preconditions.checkArgument( scanTask.file().lowerBounds() != null && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, - "Missing statistics in file = %s for columnId = %s", - scanTask.file(), - (Object) Integer.valueOf(eventTimeFieldId)); + "Missing statistics for columnName = %s, columnId = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); return timeUnit.toMillis( Conversions.fromByteBuffer( Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java index 9ae18c4cff7c..02ef57d344b1 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -47,7 +47,7 @@ public void emitRecord( long newWatermark = timeExtractor.extractWatermark(split); if (newWatermark < watermark) { LOG.info( - "previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", watermark, newWatermark, lastSplitId, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index badae552723f..bf482ae50e9b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -26,11 +26,13 @@ import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; +import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; @@ -49,6 +51,7 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.util.CloseableIterator; @@ -61,6 +64,7 @@ import org.apache.iceberg.flink.RowDataConverter; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.awaitility.Awaitility; @@ -94,6 +98,41 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { new HadoopTableResource( TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

    + *
  • - Ordering of the splits + *
  • - Emitting of watermarks + *
  • - Firing windows based on watermarks + *
+ * + *

The test generates 4 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
  • - Split 4 - Watermark 6 min + *
+ * + *

Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

    + *
  • - Split 2, 3 + *
  • - Split 4 + *
  • - Split 1 + *
+ * + *

As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ @Test public void testWindowing() throws Exception { GenericAppenderHelper dataAppender = appender(); @@ -105,27 +144,32 @@ public void testWindowing() throws Exception { // - File 2 - First records (Watermark 0) // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - File 3 - Parallel write for the first records (Watermark 60000) - // - Split 1 - 2 records (1, "file_3-recordTs_1"), (3, "file_3-recordTs_3") - List batch = ImmutableList.of(generateRecord(100, "100"), generateRecord(103, "103")); + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); expectedRecords.addAll(batch); dataAppender.appendToTable(batch); batch = Lists.newArrayListWithCapacity(100); for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch.add(generateRecord(i % 5, "file_2-recordTs_" + i)); + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); } expectedRecords.addAll(batch); dataAppender.appendToTable(batch); batch = ImmutableList.of( - generateRecord(1, "file_3-recordTs_1"), generateRecord(3, "file_3-recordTs_3")); + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); expectedRecords.addAll(batch); dataAppender.appendToTable(batch); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); + env.setParallelism(1); DataStream stream = env.fromSource( @@ -146,23 +190,66 @@ public void testWindowing() throws Exception { @Override public void apply( TimeWindow window, Iterable values, Collector out) { - // Just print all the data to confirm everything has arrived - values.forEach(out::collect); + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); } }); try (CloseableIterator resultIterator = windowed.collectAsync()) { env.executeAsync("Iceberg Source Windowing Test"); + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + // Write data so the windows containing test data are closed - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); - dataAppender.appendToTable(ImmutableList.of(generateRecord(1500, "last-record"))); + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); - assertRecords(resultIterator, expectedRecords); + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); } } + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

    + *
  • - Emitting of watermarks + *
  • - Watermark alignment + *
+ * + *

The test generates 3 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
+ * + * The splits are read in the following order: + * + *
    + *
  • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
  • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
+ * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

The status of the watermark alignment is checked by the alignment related metrics. + * + *

Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ @Test public void testThrottling() throws Exception { GenericAppenderHelper dataAppender = appender(); @@ -173,15 +260,14 @@ public void testThrottling() throws Exception { // - File 2 - First records (Watermark 0) // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - List batch; - batch = + List batch = ImmutableList.of( generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); dataAppender.appendToTable(batch); batch = Lists.newArrayListWithCapacity(100); for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch.add(generateRecord(i % 5, "file_2-recordTs_" + i)); + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); } dataAppender.appendToTable(batch); @@ -202,7 +288,7 @@ public void testThrottling() throws Exception { TypeInformation.of(RowData.class)); try (CloseableIterator resultIterator = stream.collectAsync()) { - JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); // Check that the read the non-blocked data // The first RECORD_NUM_FOR_2_SPLITS should be read @@ -223,22 +309,22 @@ public void testThrottling() throws Exception { // Add some old records with 2 splits, so even if the blocked gets one split, the other reader // one gets one as well - batch = + List newBatch1 = ImmutableList.of( generateRecord(15, "file_3-recordTs_15"), generateRecord(16, "file_3-recordTs_16"), generateRecord(17, "file_3-recordTs_17")); - dataAppender.appendToTable(batch); - batch = + List newBatch2 = ImmutableList.of( generateRecord(15, "file_4-recordTs_15"), generateRecord(16, "file_4-recordTs_16"), generateRecord(17, "file_4-recordTs_17")); - dataAppender.appendToTable(batch); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); // The records received will highly depend on scheduling // We minimally get 3 records from the non-blocked reader // We might get 1 record from the blocked reader (as part of the previous batch - - // file1-record-ts) + // file_1-recordTs_100) // We might get 3 records form the non-blocked reader if it gets both new splits waitForRecords(resultIterator, 3); @@ -284,43 +370,23 @@ protected Record generateRecord(int minutes, String str) { return record; } - protected void assertRecords(CloseableIterator iterator, List expectedRecords) - throws Exception { - - Set received = Sets.newHashSetWithExpectedSize(expectedRecords.size()); - - assertThat( - CompletableFuture.supplyAsync( - () -> { - int count = 0; - while (count < expectedRecords.size() && iterator.hasNext()) { - received.add(iterator.next()); - count++; - } - - if (count < expectedRecords.size()) { - throw new IllegalStateException( - String.format("Fail to get %d records.", expectedRecords.size())); - } - - return true; - })) - .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); - + protected void assertRecords( + Collection expectedRecords, CloseableIterator iterator) throws Exception { Set expected = expectedRecords.stream() .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) .collect(Collectors.toSet()); - Assert.assertEquals(expected, received); + Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); } - protected void waitForRecords(CloseableIterator iterator, int num) { + protected Set waitForRecords(CloseableIterator iterator, int num) { + Set received = Sets.newHashSetWithExpectedSize(num); assertThat( CompletableFuture.supplyAsync( () -> { int count = 0; while (count < num && iterator.hasNext()) { - iterator.next(); + received.add(iterator.next()); count++; } @@ -331,6 +397,8 @@ protected void waitForRecords(CloseableIterator iterator, int num) { return true; })) .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + return received; } private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { @@ -350,6 +418,13 @@ private GenericAppenderHelper appender() { sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); } + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { @Override public long extractTimestamp(RowData element, long recordTimestamp) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index f28677ca9d6a..090b304942c6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -43,15 +43,13 @@ public void testEmptyInitialization() { @Test public void testStaticEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); - assigner.onUnassignedSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1)); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -66,15 +64,12 @@ public void testContinuousEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - List splits1 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits1 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits2 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 2, 1)); + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -125,5 +120,11 @@ protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Assert.assertEquals(splitCount, stateBeforeGet.size()); } + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index 8b9e132e0e22..e78634e6b873 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; @@ -40,9 +39,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); Assertions.assertThatThrownBy( - () -> - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Please use 'split-open-file-cost'"); @@ -52,8 +49,7 @@ public void testMultipleFilesInAnIcebergSplit() { @Test public void testSplitSort() throws Exception { SplitAssigner assigner = splitAssigner(); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + List splits = createSplits(5, 1, "2"); assigner.onDiscoveredSplits(splits.subList(3, 5)); assigner.onDiscoveredSplits(splits.subList(0, 1)); @@ -76,7 +72,7 @@ public void testSerializable() { Assert.assertNotNull(comparator); } - protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * 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.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 154a89b56f59..afe8a5d0152c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -160,10 +160,11 @@ public void testEmptyStatistics() throws IOException { Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); // Create an extractor for a column we do not have statistics - ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(10); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Missing statistics in file"); + .hasMessageContaining("Missing statistics for column"); } private IcebergSourceSplit split(int id) throws IOException { From 3f823f3ba020e2f7e5c470fcc92426af37ee5103 Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Wed, 22 Nov 2023 11:46:10 +0100 Subject: [PATCH 14/15] Final comments and flaky fix --- .../reader/ColumnStatsWatermarkExtractor.java | 2 +- ...gSourceFailoverWithWatermarkExtractor.java | 8 +- ...stIcebergSourceWithWatermarkExtractor.java | 117 ++++++++++-------- 3 files changed, 72 insertions(+), 55 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java index 44a1f9c3e47e..4bb6f0a98c4c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -84,7 +84,7 @@ public long extractWatermark(IcebergSourceSplit split) { Preconditions.checkArgument( scanTask.file().lowerBounds() != null && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, - "Missing statistics for columnName = %s, columnId = %s in file = %s", + "Missing statistics for column name = %s in file = %s", eventTimeFieldName, eventTimeFieldId, scanTask.file()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index d99348b9852c..f7dc931c506c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -84,23 +84,23 @@ record -> { @Override protected void assertRecords(Table table, List expectedRecords, Duration timeout) throws Exception { - List expectedNormalized = convertTimestampField(expectedRecords); + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); Awaitility.await("expected list of records should be produced") .atMost(timeout) .untilAsserted( () -> { SimpleDataUtil.equalsRecords( expectedNormalized, - convertTimestampField(SimpleDataUtil.tableRecords(table)), + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), table.schema()); SimpleDataUtil.assertRecordsEqual( expectedNormalized, - convertTimestampField(SimpleDataUtil.tableRecords(table)), + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), table.schema()); }); } - private List convertTimestampField(List records) { + private List convertLocalDateTimeToMilli(List records) { return records.stream() .peek( r -> { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index bf482ae50e9b..0bb2eb7766e9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -31,8 +31,9 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; @@ -64,8 +65,9 @@ import org.apache.iceberg.flink.RowDataConverter; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.awaitility.Awaitility; import org.junit.Assert; @@ -79,6 +81,7 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; private static final String SOURCE_NAME = "IcebergSource"; private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -173,47 +176,59 @@ public void testWindowing() throws Exception { DataStream stream = env.fromSource( - sourceBuilder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(), + source(), WatermarkStrategy.noWatermarks() .withTimestampAssigner(new RowDataTimestampAssigner()), SOURCE_NAME, TypeInformation.of(RowData.class)); - DataStream windowed = - stream - .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) - .apply( - new AllWindowFunction() { - @Override - public void apply( - TimeWindow window, Iterable values, Collector out) { - // Emit RowData which contains the window start time, and the record count in - // that window - AtomicLong count = new AtomicLong(0); - values.forEach(a -> count.incrementAndGet()); - out.collect(row(window.getStart(), count.get())); - } - }); - - try (CloseableIterator resultIterator = windowed.collectAsync()) { - env.executeAsync("Iceberg Source Windowing Test"); - - // Wait for the 2 first windows from File 2 and File 3 - Assert.assertEquals( - ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), - waitForRecords(resultIterator, 2)); - // Write data so the windows containing test data are closed - dataAppender.appendToTable( - dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), - dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); - - // Wait for last test record window from File 1 - Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); - } + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); } /** @@ -277,11 +292,7 @@ public void testThrottling() throws Exception { DataStream stream = env.fromSource( - sourceBuilder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(), + source(), WatermarkStrategy.noWatermarks() .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), SOURCE_NAME, @@ -299,7 +310,8 @@ public void testThrottling() throws Exception { // (100 min - 20 min - 0 min) // Also this validates that the WatermarkAlignment is working Awaitility.await() - .atMost(120, TimeUnit.SECONDS) + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) .until( () -> findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) @@ -331,31 +343,36 @@ public void testThrottling() throws Exception { // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 // min - 15 min) Awaitility.await() - .atMost(120, TimeUnit.SECONDS) + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); // Add some new records which should unblock the throttled reader batch = ImmutableList.of( - generateRecord(110, "file_5-recordTs_110"), - generateRecord(111, "file_5-recordTs_111")); + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); dataAppender.appendToTable(batch); // We should get all the records at this point waitForRecords(resultIterator, 6); // Wait for the new drift to decrease below the allowed drift to signal the normal state Awaitility.await() - .atMost(120, TimeUnit.SECONDS) + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); } } - protected IcebergSource.Builder sourceBuilder() { + protected IcebergSource source() { return IcebergSource.builder() .tableLoader(sourceTableResource.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) - .splitSize(100L); + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(2)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); } protected Record generateRecord(int minutes, String str) { From d6dc3a04e2be3b369b8ccba9b88427ce5f6e425b Mon Sep 17 00:00:00 2001 From: Peter Vary Date: Thu, 23 Nov 2023 09:43:07 +0100 Subject: [PATCH 15/15] Javadoc change --- .../apache/iceberg/flink/source/IcebergSource.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index cf745a35c345..179253cb3a18 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -445,10 +445,13 @@ public Builder setAll(Map properties) { } /** - * Emits watermarks once per split based on the file statistics for the given split. The - * generated watermarks are also used for ordering the splits for read. Accepted column types - * are timestamp/timestamptz/long. For long columns consider setting {@link - * #watermarkTimeUnit(TimeUnit)}. + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. */ public Builder watermarkColumn(String columnName) { Preconditions.checkArgument(