-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[spark] Enable TagCreationMode#batch for spark writer #5185
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,118 @@ | ||
/* | ||
* 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.tag; | ||
|
||
import org.apache.paimon.CoreOptions; | ||
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 java.time.Instant; | ||
import java.time.LocalDateTime; | ||
import java.time.ZoneId; | ||
import java.time.format.DateTimeFormatter; | ||
import java.util.List; | ||
|
||
/** Tag creation for batch mode. */ | ||
public class TagBatchCreation { | ||
|
||
private static final String BATCH_WRITE_TAG_PREFIX = "batch-write-"; | ||
private final FileStoreTable table; | ||
private final CoreOptions options; | ||
private final TagManager tagManager; | ||
private final SnapshotManager snapshotManager; | ||
private final TagDeletion tagDeletion; | ||
|
||
public TagBatchCreation(FileStoreTable table) { | ||
this.table = table; | ||
this.snapshotManager = table.snapshotManager(); | ||
this.tagManager = table.tagManager(); | ||
this.tagDeletion = table.store().newTagDeletion(); | ||
this.options = table.coreOptions(); | ||
} | ||
|
||
public void createTag() { | ||
Snapshot snapshot = snapshotManager.latestSnapshot(); | ||
if (snapshot == null) { | ||
return; | ||
} | ||
Instant instant = Instant.ofEpochMilli(snapshot.timeMillis()); | ||
LocalDateTime localDateTime = LocalDateTime.ofInstant(instant, ZoneId.systemDefault()); | ||
String tagName = | ||
options.tagBatchCustomizedName() != null | ||
? options.tagBatchCustomizedName() | ||
: BATCH_WRITE_TAG_PREFIX | ||
+ localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd")); | ||
try { | ||
// If the tag already exists, delete the tag | ||
tagManager.deleteTag( | ||
tagName, tagDeletion, snapshotManager, table.store().createTagCallbacks()); | ||
// Create a new tag | ||
tagManager.createTag( | ||
snapshot, | ||
tagName, | ||
table.coreOptions().tagDefaultTimeRetained(), | ||
table.store().createTagCallbacks(), | ||
false); | ||
// Expire the tag | ||
expireTag(); | ||
} catch (Exception e) { | ||
if (tagManager.tagExists(tagName)) { | ||
tagManager.deleteTag( | ||
tagName, tagDeletion, snapshotManager, table.store().createTagCallbacks()); | ||
} | ||
} | ||
} | ||
|
||
private void expireTag() { | ||
Integer tagNumRetainedMax = options.tagNumRetainedMax(); | ||
if (tagNumRetainedMax != null) { | ||
if (snapshotManager.latestSnapshot() == null) { | ||
return; | ||
} | ||
long tagCount = tagManager.tagCount(); | ||
|
||
while (tagCount > tagNumRetainedMax) { | ||
for (List<String> tagNames : tagManager.tags().values()) { | ||
if (tagCount - tagNames.size() >= tagNumRetainedMax) { | ||
tagManager.deleteAllTagsOfOneSnapshot( | ||
tagNames, tagDeletion, snapshotManager); | ||
tagCount = tagCount - tagNames.size(); | ||
} else { | ||
List<String> sortedTagNames = tagManager.sortTagsOfOneSnapshot(tagNames); | ||
for (String toBeDeleted : sortedTagNames) { | ||
tagManager.deleteTag( | ||
toBeDeleted, | ||
tagDeletion, | ||
snapshotManager, | ||
table.store().createTagCallbacks()); | ||
tagCount--; | ||
if (tagCount == tagNumRetainedMax) { | ||
break; | ||
} | ||
} | ||
break; | ||
} | ||
} | ||
} | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,13 +19,14 @@ | |
package org.apache.paimon.spark.commands | ||
|
||
import org.apache.paimon.CoreOptions | ||
import org.apache.paimon.CoreOptions.DYNAMIC_PARTITION_OVERWRITE | ||
import org.apache.paimon.CoreOptions.{DYNAMIC_PARTITION_OVERWRITE, TagCreationMode} | ||
import org.apache.paimon.options.Options | ||
import org.apache.paimon.partition.actions.PartitionMarkDoneAction | ||
import org.apache.paimon.spark._ | ||
import org.apache.paimon.spark.schema.SparkSystemColumns | ||
import org.apache.paimon.table.FileStoreTable | ||
import org.apache.paimon.table.sink.CommitMessage | ||
import org.apache.paimon.tag.TagBatchCreation | ||
import org.apache.paimon.utils.{InternalRowPartitionComputer, PartitionPathUtils, TypeUtils} | ||
|
||
import org.apache.spark.internal.Logging | ||
|
@@ -83,10 +84,19 @@ case class WriteIntoPaimonTable( | |
val commitMessages = writer.write(data) | ||
writer.commit(commitMessages) | ||
|
||
markDoneIfNeeded(commitMessages) | ||
preFinish(commitMessages) | ||
Seq.empty | ||
} | ||
|
||
private def preFinish(commitMessages: Seq[CommitMessage]): Unit = { | ||
val coreOptions = table.coreOptions(); | ||
if (coreOptions.tagCreationMode() == TagCreationMode.BATCH) { | ||
val tagCreation = new TagBatchCreation(table) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe a little more concise : There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fixed |
||
tagCreation.createTag() | ||
} | ||
markDoneIfNeeded(commitMessages) | ||
} | ||
|
||
private def markDoneIfNeeded(commitMessages: Seq[CommitMessage]): Unit = { | ||
val coreOptions = table.coreOptions() | ||
if (coreOptions.toConfiguration.get(CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT)) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this option is used several times, I think it's ok here There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My mistake. |
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Waring-Logs should be printed here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fixed