Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

This page is meant as a template for writing a FLIP. To create a FLIP choose Tools->Copy on this page and modify with your content and replace the heading with the next FLIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state:  ["Under Discussion"] Accepted

Discussion thread:  here (<- link to https://mail-archiveslists.apache.org/mod_mbox/flink-dev/)
thread/n6nsvbwhs5kwlj5kjgv24by2tk5mh9xd

VOTE thread: JIRAhere (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)lists.apache.org/thread/8c0wlp72kq0dhcbpy08nl1kb28q17kv3

JIRA:

Jira
serverASF JIRA
columnIdsissuekey,summary,issuetype,created,updated,duedate,assignee,reporter,customfield_12311032,customfield_12311037,customfield_12311022,customfield_12311027,priority,status,resolution
columnskey,summary,type,created,updated,due,assignee,reporter,Priority,Priority,Priority,Priority,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-32580

Released: 1.18

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

Code Block
languagejava
/**
 * A {@link StagedTable} for atomic semantics using a two-phase commit protocol, combined with
 * {@link JobStatusHook} for atomic CTAS. {@link StagedTable} will be a member variable of
 * CtasJobStatusHook and can be serialized;
 *
 * <p>CtasJobStatusHook#onCreated will call the begin method of StagedTable;
 * CtasJobStatusHook#onFinished will call the commit method of StagedTable;
 * CtasJobStatusHook#onFailed and CtasJobStatusHook#onCanceled will call the abort method of StagedTable;
 */
@PublicEvolving
public interface StagedTable extends Serializable {

    /**
     * This method will be called when the job is started. Similar to what it means to open a
     * transaction in a relational database; In Flink's atomic CTAS scenario, it is used to do some
     * initialization work; For example, initializing the client of the underlying service, the tmp
     * path of the underlying storage, or even call the start transaction API of the underlying
     * service, etc.
     */
    void begin();

    /**
     * This method will be called when the job is succeeds. Similar to what it means to commit the
     * transaction in a relational database; In Flink's atomic CTAS scenario, it is used to do some
     * data visibility related work; For example, moving the underlying data to the target
     * directory, writing buffer data to the underlying storage service, or even call the commit
     * transaction API of the underlying service, etc.
     */
    void commit();

    /**
     * This method will be called when the job is failed or canceled. Similar to what it means to
     * rollback the transaction in a relational database; In Flink's atomic CTAS scenario, it is
     * used to do some data cleaning; For example, delete the data in tmp directory, delete the
     * temporary data in the underlying storage service, or even call the rollback transaction API
     * of the underlying service, etc.
     */
    void abort();
}

TableConfigOptions

Add table.ctas.atomicity-enabled option to allow users to enable atomicity when using create table as select syntax.

Code Block
languagejava
@PublicEvolving
public class TableConfigOptions {
    @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
    public static final ConfigOption<Boolean> TABLE_CTAS_ATOMICITY_ENABLED =
            key("table.ctas.atomicity-enabled")
                    .booleanType()
                    .defaultValue(false)
                    .withDescription(
                            "Specifies if the create table as select operation is executed atomically. "
                                  + "By default, the operation is non-atomic. The target table is created in Client side, and it will not be dropped even though the job fails or is cancelled. "
                                  + "If set this option to true and DynamicTableSink implements the SupportsStaging interface, the targetcreate table as select operation is createdexpected to inbe JMexecuted side, it also will be dropped when the job fails or is cancelledatomically, "
                                  + "the behavior of which depends on the actual DynamicTableSink.");
}

Proposed Changes

First we need to have a Table interface that can be combined with the abstract transaction capability, so we introduce StagedTable, which can perform start transaction, commit transaction, and abort transaction operations.

...

Identification of atomic CTAS

Normally, in stream mode, we consider the job to be LONG RUNNING, and even if it fails, it needs to resume afterwards, so atomic CTAS semantics are usually not needed.

In addition, there are probably many flink jobs that already use At present, there may be a large number of flink jobs using non-atomic CTAS functions, especially Stream jobs, in order to ensure the consistency of flink behavior, and to give the user maximum flexibility, in time DynamicTableSink implements the SupportsStaging interface, users can still choose non-atomic implementation according to business needs.

...

Code Block
languagejava
Optional<DynamicTableSink> dynamicTableSinkOptional =
        getDynamicTableSink(
                catalogTable,
                tableIdentifier,
                createTableOperation.isTemporary(),
                catalogManager.getCatalog(catalogName)); 
if (tableConfig.get(TableConfigOptions.TABLE_CTAS_ATOMICITY_ENABLED)
        && dynamicTableSinkOptional.isPresent()
        && dynamicTableSinkOptional.get() instanceof SupportsStaging) {
     DynamicTableSink DynamicTableSink dynamicTableSink = dynamicTableSinkOptional.get();
    StagedTable stagedTable =
            ((SupportsStaging) dynamicTableSink)
                    .applyStaging(
                            new SupportsStaging.StagingContext() {
                                @Override
                                public SupportsStaging.StagingPurpose
                                        getStagingPurpose() {
                                    if (createTableOperation
                                            .isIgnoreIfExists()) {
                                        return SupportsStaging.StagingPurpose
                                                .CREATE_TABLE_AS_IF_NOT_EXISTS;
                                    }
                                    return SupportsStaging.StagingPurpose
                                            .CREATE_TABLE_AS;
                                }
                            });
    CtasJobStatusHook ctasJobStatusHook = new CtasJobStatusHook(stagedTable);
    mapOperations.add(
            ctasOperation.toStagedSinkModifyOperation(
                    createTableOperation.getTableIdentifier(),
                    catalogTable,
                    ctasCatalog,
                    dynamicTableSink));
    jobStatusHookList.add(ctasJobStatusHook);
} else {
    // execute CREATE TABLE first for non-atomic CTAS statements
    executeInternal(ctasOperation.getCreateTableOperation());
    mapOperations.add(ctasOperation.toSinkModifyOperation(catalogManager));
}

...