Skip to content

Commit

Permalink
[Kernel]Generalize the actions after commit(like checkpoint) by intro…
Browse files Browse the repository at this point in the history
…ducing post commit action to kernel (#4115)

<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [ ] Spark
- [ ] Standalone
- [ ] Flink
- [X] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->
This PR doesn't make any functional changes, but abstract checkpoint
into post commit action. This is prepared adding more post commit
actions such as CRC write (#4116)
## How was this patch tested?
Existing unit test, manual test using
delta/kernel/examples/run-kernel-examples.py --use-local

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
No
  • Loading branch information
huan233usc authored Feb 6, 2025
1 parent 2d07216 commit 685379c
Show file tree
Hide file tree
Showing 7 changed files with 160 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,16 @@

import java.io.IOException;
import java.util.*;
import java.util.concurrent.CompletableFuture;

import org.apache.commons.cli.Options;

import io.delta.kernel.*;
import io.delta.kernel.data.FilteredColumnarBatch;
import io.delta.kernel.data.Row;
import io.delta.kernel.expressions.Literal;
import io.delta.kernel.hook.PostCommitHook;
import io.delta.kernel.hook.PostCommitHook.PostCommitHookType;
import io.delta.kernel.utils.*;
import static io.delta.kernel.examples.utils.Utils.parseArgs;

Expand Down Expand Up @@ -409,11 +412,17 @@ public void insertWithOptionalCheckpoint(String tablePath) throws IOException {
// for every 10 versions.
for (int i = 0; i < 12; i++) {
TransactionCommitResult commitResult = insertDataIntoUnpartitionedTable(tablePath);
if (commitResult.isReadyForCheckpoint()) {
for(PostCommitHook hook: commitResult.getPostCommitHooks())
// Checkpoint the table
Table.forPath(engine, tablePath).checkpoint(engine, commitResult.getVersion());
didCheckpoint = true;
}
didCheckpoint = didCheckpoint || CompletableFuture.supplyAsync(() -> {
// run the code async
try{
hook.threadSafeInvoke(engine);
} catch (IOException e) {
return false;
}
return hook.getType().equals(PostCommitHookType.CHECKPOINT);
}).join(); // wait async finish.
}

if (!didCheckpoint) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@

import io.delta.kernel.annotation.Evolving;
import io.delta.kernel.engine.Engine;
import io.delta.kernel.hook.PostCommitHook;
import io.delta.kernel.utils.CloseableIterable;
import java.util.List;

/**
* Contains the result of a successful transaction commit. Returned by {@link
Expand All @@ -28,11 +30,11 @@
@Evolving
public class TransactionCommitResult {
private final long version;
private final boolean isReadyForCheckpoint;
private final List<PostCommitHook> postCommitHooks;

public TransactionCommitResult(long version, boolean isReadyForCheckpoint) {
public TransactionCommitResult(long version, List<PostCommitHook> postCommitHooks) {
this.version = version;
this.isReadyForCheckpoint = isReadyForCheckpoint;
this.postCommitHooks = postCommitHooks;
}

/**
Expand All @@ -45,13 +47,19 @@ public long getVersion() {
}

/**
* Is the table ready for checkpoint (i.e. there are enough commits since the last checkpoint)? If
* yes the connector can choose to checkpoint as the version the transaction is committed as using
* {@link Table#checkpoint(Engine, long)}
* Operations for connector to trigger post-commit.
*
* @return Is the table ready for checkpointing?
* <p>Usage:
*
* <ul>
* <li>Async: Call {@link PostCommitHook#threadSafeInvoke(Engine)} in separate thread.
* <li>Sync: Direct call {@link PostCommitHook#threadSafeInvoke(Engine)} and block until
* operation ends.
* </ul>
*
* @return list of post-commit operations
*/
public boolean isReadyForCheckpoint() {
return isReadyForCheckpoint;
public List<PostCommitHook> getPostCommitHooks() {
return postCommitHooks;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Copyright (2025) The Delta Lake Project Authors.
*
* Licensed 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 io.delta.kernel.hook;

import io.delta.kernel.engine.Engine;
import java.io.IOException;

/**
* A hook for executing operation after a transaction commit. Hooks are added in the Transaction and
* engine need to invoke the hook explicitly for executing the operation. Supported operations are
* listed in {@link PostCommitHookType}.
*/
public interface PostCommitHook {

enum PostCommitHookType {
/**
* Writes a new checkpoint at the version committed by the transaction. This hook is present
* when the table is ready for checkpoint according to its configured checkpoint interval. To
* perform this operation, reading previous checkpoint + logs is required to construct a new
* checkpoint, with latency scaling based on log size (typically seconds to minutes).
*/
CHECKPOINT
}

/** Invokes the post commit operation whose implementation must be thread safe. */
void threadSafeInvoke(Engine engine) throws IOException;

PostCommitHookType getType();
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,11 @@
import io.delta.kernel.engine.Engine;
import io.delta.kernel.exceptions.ConcurrentWriteException;
import io.delta.kernel.expressions.Column;
import io.delta.kernel.hook.PostCommitHook;
import io.delta.kernel.internal.actions.*;
import io.delta.kernel.internal.data.TransactionStateRow;
import io.delta.kernel.internal.fs.Path;
import io.delta.kernel.internal.hook.CheckpointHook;
import io.delta.kernel.internal.metrics.TransactionMetrics;
import io.delta.kernel.internal.metrics.TransactionReportImpl;
import io.delta.kernel.internal.replay.ConflictChecker;
Expand Down Expand Up @@ -354,7 +356,12 @@ private TransactionCommitResult doCommit(
"Write file actions to JSON log file `%s`",
FileNames.deltaFile(logPath, commitAsVersion));

return new TransactionCommitResult(commitAsVersion, isReadyForCheckpoint(commitAsVersion));
List<PostCommitHook> postCommitHooks = new ArrayList<>();
if (isReadyForCheckpoint(commitAsVersion)) {
postCommitHooks.add(new CheckpointHook(dataPath, commitAsVersion));
}

return new TransactionCommitResult(commitAsVersion, postCommitHooks);
} catch (FileAlreadyExistsException e) {
throw e;
} catch (IOException ioe) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* Copyright (2025) The Delta Lake Project Authors.
*
* Licensed 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 io.delta.kernel.internal.hook;

import io.delta.kernel.Table;
import io.delta.kernel.engine.Engine;
import io.delta.kernel.hook.PostCommitHook;
import io.delta.kernel.internal.fs.Path;
import java.io.IOException;

/** Write a new checkpoint at the version committed by the txn. */
public class CheckpointHook implements PostCommitHook {

private final Path tablePath;
private final long checkpointVersion;

public CheckpointHook(Path tablePath, long checkpointVersion) {
this.tablePath = tablePath;
this.checkpointVersion = checkpointVersion;
}

@Override
public void threadSafeInvoke(Engine engine) throws IOException {
Table.forPath(engine, tablePath.toString()).checkpoint(engine, checkpointVersion);
}

@Override
public PostCommitHookType getType() {
return PostCommitHookType.CHECKPOINT;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,15 @@ import io.delta.kernel.internal.util.FileNames
import io.delta.kernel.internal.util.Utils.singletonCloseableIterator
import io.delta.kernel.internal.{SnapshotImpl, TableConfig, TableImpl}
import io.delta.kernel.utils.FileStatus
import io.delta.kernel.{Meta, Operation, Table, Transaction, TransactionBuilder, TransactionCommitResult}
import io.delta.kernel.data.{ColumnarBatch, ColumnVector, FilteredColumnarBatch, Row}
import io.delta.kernel.{
Meta,
Operation,
Table,
Transaction,
TransactionBuilder,
TransactionCommitResult
}
import io.delta.kernel.data.{ColumnVector, ColumnarBatch, FilteredColumnarBatch, Row}
import io.delta.kernel.defaults.internal.data.DefaultColumnarBatch
import io.delta.kernel.expressions.Literal
import io.delta.kernel.expressions.Literal.ofInt
Expand All @@ -39,6 +46,7 @@ import io.delta.kernel.types.StructType
import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable}
import io.delta.kernel.utils.CloseableIterator
import io.delta.kernel.Operation.CREATE_TABLE
import io.delta.kernel.hook.PostCommitHook.PostCommitHookType
import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -139,10 +147,14 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils {
tablePath: String,
result: TransactionCommitResult,
expSize: Long): Unit = {
if (result.isReadyForCheckpoint) {
Table.forPath(engine, tablePath).checkpoint(engine, result.getVersion)
verifyLastCheckpointMetadata(tablePath, checkpointAt = result.getVersion, expSize)
}
result.getPostCommitHooks.forEach(
hook => {
if (hook.getType == PostCommitHookType.CHECKPOINT) {
hook.threadSafeInvoke(engine)
verifyLastCheckpointMetadata(tablePath, checkpointAt = result.getVersion, expSize)
}
}
)
}

/**
Expand Down Expand Up @@ -399,7 +411,7 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils {
expVersion: Long,
expIsReadyForCheckpoint: Boolean): Unit = {
assert(result.getVersion === expVersion)
assert(result.isReadyForCheckpoint === expIsReadyForCheckpoint)
assertCheckpointReadiness(result, expIsReadyForCheckpoint)
}

def verifyTableProperties(
Expand All @@ -421,4 +433,16 @@ trait DeltaTableWriteSuiteBase extends AnyFunSuite with TestUtils {
builder.append("]")
checkAnswer(resultProperties, Seq(builder.toString()).map(TestRow(_)))
}

def assertCheckpointReadiness(
txnResult: TransactionCommitResult,
isReadyForCheckpoint: Boolean): Unit = {
assert(
txnResult.getPostCommitHooks
.stream()
.anyMatch(
hook => hook.getType == PostCommitHookType.CHECKPOINT
) === isReadyForCheckpoint
)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import io.delta.kernel.engine.Engine
import io.delta.kernel.exceptions._
import io.delta.kernel.expressions.Literal
import io.delta.kernel.expressions.Literal._
import io.delta.kernel.hook.PostCommitHook.PostCommitHookType
import io.delta.kernel.internal.checkpoints.CheckpointerSuite.selectSingleElement
import io.delta.kernel.internal.util.SchemaUtils.casePreservingPartitionColNames
import io.delta.kernel.internal.{SnapshotImpl, TableConfig}
Expand Down Expand Up @@ -131,7 +132,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa
val txnResult = txn.commit(engine, emptyIterable())

assert(txnResult.getVersion === 0)
assert(!txnResult.isReadyForCheckpoint)
assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false)

verifyCommitInfo(tablePath = tablePath, version = 0)
verifyWrittenContent(tablePath, testSchema, Seq.empty)
Expand Down Expand Up @@ -350,7 +351,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa
val txnResult = txn.commit(engine, emptyIterable())

assert(txnResult.getVersion === 0)
assert(!txnResult.isReadyForCheckpoint)
assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false)

verifyCommitInfo(tablePath, version = 0, Seq("Part1", "part2"))
verifyWrittenContent(tablePath, schema, Seq.empty)
Expand All @@ -368,7 +369,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa
val txnResult = txn.commit(engine, emptyIterable())

assert(txnResult.getVersion === 0)
assert(!txnResult.isReadyForCheckpoint)
assertCheckpointReadiness(txnResult, isReadyForCheckpoint = false)

verifyCommitInfo(tablePath, version = 0)
verifyWrittenContent(tablePath, schema, Seq.empty)
Expand Down

0 comments on commit 685379c

Please sign in to comment.