forked from apache/iceberg
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Flink: Backport apache#8553 to v1.15, v1.16
- Loading branch information
Peter Vary
committed
Nov 24, 2023
1 parent
21b3eea
commit 27333e3
Showing
32 changed files
with
2,460 additions
and
86 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
98 changes: 98 additions & 0 deletions
98
...k/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,98 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.source.reader; | ||
|
||
import java.io.Serializable; | ||
import java.util.Comparator; | ||
import 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.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; | ||
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. | ||
*/ | ||
@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 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 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, String eventTimeFieldName) { | ||
this.eventTimeFieldId = eventTimeFieldId; | ||
this.eventTimeFieldName = eventTimeFieldName; | ||
this.timeUnit = TimeUnit.MICROSECONDS; | ||
} | ||
|
||
/** | ||
* Get the watermark for a split using column statistics. | ||
* | ||
* @param split The split | ||
* @return The watermark | ||
* @throws IllegalArgumentException if there is no statistics for the column | ||
*/ | ||
@Override | ||
public long extractWatermark(IcebergSourceSplit split) { | ||
return split.task().files().stream() | ||
.map( | ||
scanTask -> { | ||
Preconditions.checkArgument( | ||
scanTask.file().lowerBounds() != null | ||
&& scanTask.file().lowerBounds().get(eventTimeFieldId) != null, | ||
"Missing statistics for column name = %s in file = %s", | ||
eventTimeFieldName, | ||
eventTimeFieldId, | ||
scanTask.file()); | ||
return timeUnit.toMillis( | ||
Conversions.fromByteBuffer( | ||
Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); | ||
}) | ||
.min(Comparator.comparingLong(l -> l)) | ||
.get(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
40 changes: 40 additions & 0 deletions
40
...flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* 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.annotation.Internal; | ||
import org.apache.flink.connector.base.source.reader.RecordEmitter; | ||
import org.apache.iceberg.flink.source.split.IcebergSourceSplit; | ||
|
||
@Internal | ||
@FunctionalInterface | ||
public interface SerializableRecordEmitter<T> | ||
extends RecordEmitter<RecordAndPosition<T>, T, IcebergSourceSplit>, Serializable { | ||
static <T> SerializableRecordEmitter<T> defaultEmitter() { | ||
return (element, output, split) -> { | ||
output.collect(element.record()); | ||
split.updatePosition(element.fileOffset(), element.recordOffset()); | ||
}; | ||
} | ||
|
||
static <T> SerializableRecordEmitter<T> emitterWithWatermark(SplitWatermarkExtractor extractor) { | ||
return new WatermarkExtractorRecordEmitter<>(extractor); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.