Skip to content

Commit

Permalink
[flink] Generate tags when Flink Batch is completed
Browse files Browse the repository at this point in the history
This closes apache#2469

(cherry picked from commit 5a8b597)
  • Loading branch information
siyang.zeng committed Jan 4, 2024
1 parent 94104cf commit bf6e8d6
Show file tree
Hide file tree
Showing 8 changed files with 369 additions and 5 deletions.
1 change: 1 addition & 0 deletions docs/content/maintenance/manage-tags.md
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ Paimon supports automatic creation of tags in writing job.
You can set `'tag.automatic-creation'` to `process-time` or `watermark`:
- `process-time`: Create TAG based on the time of the machine.
- `watermark`: Create TAG based on the watermark of the Sink input.
- `batch`: In a batch processing scenario, a tag is generated after the current task is completed.

{{< hint info >}}
If you choose Watermark, you may need to specify the time zone of watermark, if watermark is not in the
Expand Down
4 changes: 2 additions & 2 deletions docs/layouts/shortcodes/generated/core_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@
<td><h5>metastore.tag-to-partition.preview</h5></td>
<td style="word-wrap: break-word;">none</td>
<td><p>Enum</p></td>
<td>Whether to preview tag of generated snapshots in metastore. This allows the Hive engine to query specific tag before creation.<br /><br />Possible values:<ul><li>"none": No automatically created tags.</li><li>"process-time": Based on the time of the machine, create TAG once the processing time passes period time plus delay.</li><li>"watermark": Based on the watermark of the input, create TAG once the watermark passes period time plus delay.</li></ul></td>
<td>Whether to preview tag of generated snapshots in metastore. This allows the Hive engine to query specific tag before creation.<br /><br />Possible values:<ul><li>"none": No automatically created tags.</li><li>"process-time": Based on the time of the machine, create TAG once the processing time passes period time plus delay.</li><li>"watermark": Based on the watermark of the input, create TAG once the watermark passes period time plus delay.</li><li>"batch": In the batch processing scenario, the tag corresponding to the current snapshot is generated after the task is completed.</li></ul></td>
</tr>
<tr>
<td><h5>num-levels</h5></td>
Expand Down Expand Up @@ -573,7 +573,7 @@
<td><h5>tag.automatic-creation</h5></td>
<td style="word-wrap: break-word;">none</td>
<td><p>Enum</p></td>
<td>Whether to create tag automatically. And how to generate tags.<br /><br />Possible values:<ul><li>"none": No automatically created tags.</li><li>"process-time": Based on the time of the machine, create TAG once the processing time passes period time plus delay.</li><li>"watermark": Based on the watermark of the input, create TAG once the watermark passes period time plus delay.</li></ul></td>
<td>Whether to create tag automatically. And how to generate tags.<br /><br />Possible values:<ul><li>"none": No automatically created tags.</li><li>"process-time": Based on the time of the machine, create TAG once the processing time passes period time plus delay.</li><li>"watermark": Based on the watermark of the input, create TAG once the watermark passes period time plus delay.</li><li>"batch": In the batch processing scenario, the tag corresponding to the current snapshot is generated after the task is completed.</li></ul></td>
</tr>
<tr>
<td><h5>tag.callback.#.param</h5></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1971,8 +1971,10 @@ public enum TagCreationMode implements DescribedEnum {
"Based on the time of the machine, create TAG once the processing time passes period time plus delay."),
WATERMARK(
"watermark",
"Based on the watermark of the input, create TAG once the watermark passes period time plus delay.");

"Based on the watermark of the input, create TAG once the watermark passes period time plus delay."),
BATCH(
"batch",
"In the batch processing scenario, the tag corresponding to the current snapshot is generated after the task is completed.");
private final String value;
private final String description;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,8 @@ private TagPreview(CoreOptions options) {
}

public static TagPreview create(CoreOptions options) {
if (options.tagToPartitionPreview() != CoreOptions.TagCreationMode.NONE) {
if (options.tagToPartitionPreview() != CoreOptions.TagCreationMode.NONE
&& options.tagToPartitionPreview() != CoreOptions.TagCreationMode.BATCH) {
return new TagPreview(options);
}
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ static TagTimeExtractor createForTagPreview(CoreOptions options) {
static TagTimeExtractor create(CoreOptions.TagCreationMode mode, CoreOptions options) {
switch (mode) {
case NONE:
case BATCH:
return null;
case PROCESS_TIME:
return new ProcessTimeExtractor();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,249 @@
/*
* 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.paimon.flink.sink;

import org.apache.paimon.Snapshot;
import org.apache.paimon.operation.TagDeletion;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.utils.SnapshotManager;
import org.apache.paimon.utils.TagManager;

import org.apache.flink.metrics.groups.OperatorMetricGroup;
import org.apache.flink.runtime.checkpoint.CheckpointOptions;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.SetupableStreamOperator;
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;

import java.time.Instant;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
import java.util.SortedMap;

/**
* Commit {@link Committable} for snapshot using the {@link CommitterOperator}. When the task is
* completed, the corresponding tag is generated.
*/
public class BatchWriteGeneratorTagOperator<CommitT, GlobalCommitT>
implements OneInputStreamOperator<CommitT, CommitT>,
SetupableStreamOperator,
BoundedOneInput {

private static final String BATCH_WRITE_TAG_PREFIX = "batch-write-";

private static final long serialVersionUID = 1L;

private final CommitterOperator<CommitT, GlobalCommitT> commitOperator;

protected final FileStoreTable table;

public BatchWriteGeneratorTagOperator(
CommitterOperator<CommitT, GlobalCommitT> commitOperator, FileStoreTable table) {
this.table = table;
this.commitOperator = commitOperator;
}

@Override
public void initializeState(StreamTaskStateInitializer streamTaskStateManager)
throws Exception {
commitOperator.initializeState(streamTaskStateManager);
}

@Override
public OperatorSnapshotFutures snapshotState(
long checkpointId,
long timestamp,
CheckpointOptions checkpointOptions,
CheckpointStreamFactory storageLocation)
throws Exception {
return commitOperator.snapshotState(
checkpointId, timestamp, checkpointOptions, storageLocation);
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
commitOperator.notifyCheckpointComplete(checkpointId);
}

@Override
public void notifyCheckpointAborted(long checkpointId) throws Exception {
commitOperator.notifyCheckpointAborted(checkpointId);
}

private void createTag() {
SnapshotManager snapshotManager = table.snapshotManager();
Snapshot snapshot = snapshotManager.latestSnapshot();
if (snapshot == null) {
return;
}
TagManager tagManager = table.tagManager();
TagDeletion tagDeletion = table.store().newTagDeletion();
Instant instant = Instant.ofEpochMilli(snapshot.timeMillis());
LocalDateTime localDateTime = LocalDateTime.ofInstant(instant, ZoneId.systemDefault());
String tagName =
BATCH_WRITE_TAG_PREFIX
+ localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd"));
try {
// If the tag already exists, delete the tag
if (tagManager.tagExists(tagName)) {
tagManager.deleteTag(tagName, tagDeletion, snapshotManager);
}
// Create a new tag
tagManager.createTag(snapshot, tagName);
// Expire the tag
expireTag();
} catch (Exception e) {
if (tagManager.tagExists(tagName)) {
tagManager.deleteTag(tagName, tagDeletion, snapshotManager);
}
}
}

private void expireTag() {
Integer tagNumRetainedMax = table.coreOptions().tagNumRetainedMax();
if (tagNumRetainedMax != null) {
SnapshotManager snapshotManager = table.snapshotManager();
if (snapshotManager.latestSnapshot() == null) {
return;
}
TagManager tagManager = table.tagManager();
TagDeletion tagDeletion = table.store().newTagDeletion();
SortedMap<Snapshot, String> tags = tagManager.tags();
if (tags.size() > tagNumRetainedMax) {
int toDelete = tags.size() - tagNumRetainedMax;
int i = 0;
for (String tag : tags.values()) {
tagManager.deleteTag(tag, tagDeletion, snapshotManager);
i++;
if (i == toDelete) {
break;
}
}
}
}
}

@Override
public void open() throws Exception {
commitOperator.open();
}

@Override
public void processElement(StreamRecord<CommitT> element) throws Exception {
commitOperator.processElement(element);
}

@Override
public void processWatermark(Watermark watermark) throws Exception {
commitOperator.processWatermark(watermark);
}

@Override
public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception {
commitOperator.processWatermarkStatus(watermarkStatus);
}

@Override
public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
commitOperator.processLatencyMarker(latencyMarker);
}

@Override
public void finish() throws Exception {
createTag();
commitOperator.finish();
}

@Override
public void close() throws Exception {
commitOperator.close();
}

@Override
public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
commitOperator.prepareSnapshotPreBarrier(checkpointId);
}

@Override
public void setKeyContextElement1(StreamRecord<?> record) throws Exception {
commitOperator.setKeyContextElement1(record);
}

@Override
public void setKeyContextElement2(StreamRecord<?> record) throws Exception {
commitOperator.setKeyContextElement2(record);
}

@Override
public OperatorMetricGroup getMetricGroup() {
return commitOperator.getMetricGroup();
}

@Override
public OperatorID getOperatorID() {
return commitOperator.getOperatorID();
}

@Override
public void setCurrentKey(Object key) {
commitOperator.setCurrentKey(key);
}

@Override
public Object getCurrentKey() {
return commitOperator.getCurrentKey();
}

@Override
public void setKeyContextElement(StreamRecord<CommitT> record) throws Exception {
commitOperator.setKeyContextElement(record);
}

@Override
public void endInput() throws Exception {
commitOperator.endInput();
}

@Override
public void setup(StreamTask containingTask, StreamConfig config, Output output) {
commitOperator.setup(containingTask, config, output);
}

@Override
public ChainingStrategy getChainingStrategy() {
return commitOperator.getChainingStrategy();
}

@Override
public void setChainingStrategy(ChainingStrategy strategy) {
commitOperator.setChainingStrategy(strategy);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.paimon.flink.sink;

import org.apache.paimon.CoreOptions.ChangelogProducer;
import org.apache.paimon.CoreOptions.TagCreationMode;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.options.MemorySize;
Expand Down Expand Up @@ -203,6 +204,13 @@ protected DataStreamSink<?> doCommit(DataStream<Committable> written, String com
table::tagManager,
() -> table.store().newTagDeletion());
}
if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH
&& table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) {
committerOperator =
new BatchWriteGeneratorTagOperator<>(
(CommitterOperator<Committable, ManifestCommittable>) committerOperator,
table);
}
SingleOutputStreamOperator<?> committed =
written.transform(
GLOBAL_COMMITTER_NAME + " : " + table.name(),
Expand Down
Loading

0 comments on commit bf6e8d6

Please sign in to comment.