tsreaper commented on code in PR #550: URL: https://github.com/apache/flink-table-store/pull/550#discussion_r1121081241
########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ Review Comment: Filter committed commit -> Filter committed snapshots ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ + default boolean filterCommitted(long commitIdentifier) { + Set<Long> filter = filterCommitted(Collections.singleton(commitIdentifier)); + return filter.contains(commitIdentifier); } - public List<ManifestCommittable> filterCommitted(List<ManifestCommittable> committables) { - return commit.filterCommitted(committables); - } - - public void commit(long identifier, List<FileCommittable> fileCommittables) { - ManifestCommittable committable = new ManifestCommittable(identifier); - for (FileCommittable fileCommittable : fileCommittables) { - committable.addFileCommittable(fileCommittable); - } - commit(Collections.singletonList(committable)); - } - - public void commit(List<ManifestCommittable> committables) { - if (overwritePartitions == null) { - for (ManifestCommittable committable : committables) { - commit.commit(committable, new HashMap<>()); - } - } else { - ManifestCommittable committable; - if (committables.size() > 1) { - throw new RuntimeException( - "Multiple committables appear in overwrite mode, this may be a bug, please report it: " - + committables); - } else if (committables.size() == 1) { - committable = committables.get(0); - } else { - // create an empty committable - // identifier is Long.MAX_VALUE, come from batch job - // TODO maybe it can be produced by CommitterOperator - committable = new ManifestCommittable(Long.MAX_VALUE); - } - commit.overwrite(overwritePartitions, committable, new HashMap<>()); - } - - if (expire != null) { - expire.expire(); - } - - if (partitionExpire != null) { - partitionExpire.expire(); - } - } - - @Override - public void close() throws Exception { - if (lock != null) { - lock.close(); - } - } + /** Filter committed commits. This method is used for failover cases. */ + Set<Long> filterCommitted(Set<Long> commitIdentifiers); + + /** + * Create a new commit. One commit may generate two snapshots, one for adding new files and the Review Comment: One commit may generate up to two snapshots ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/WriteBuilder.java: ########## @@ -0,0 +1,108 @@ +/* + * 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.flink.table.store.table.sink; + +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.data.InternalRow; +import org.apache.flink.table.store.types.RowType; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * An interface for building the {@link TableWrite} and {@link TableCommit}. + * + * <p>Example of distributed batch writing: + * + * <pre>{@code + * // 1. Create a WriteBuilder (Serializable) + * Table table = catalog.getTable(...); + * WriteBuilder builder = table.newWriteBuilder(); + * + * // 2. Write records in distributed tasks + * TableWrite write = builder.newWrite(); + * write.write(...); + * write.write(...); + * write.write(...); + * List<CommitMessage> messages = write.prepareCommit(true, 0); + * + * // 3. Collect all CommitMessages to a global node + * + * // 3.1 commit + * TableCommit commit = builder.newCommit(); + * if (commit.filterCommitted(0)) { + * // It has already been committed. In order to avoid + * // repeated committing, return directly. + * return; + * } + * commit.commit(0, allCommitMessages()); Review Comment: What does 0 mean here? Add comments. ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/WriteBuilder.java: ########## @@ -0,0 +1,108 @@ +/* + * 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.flink.table.store.table.sink; + +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.data.InternalRow; +import org.apache.flink.table.store.types.RowType; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * An interface for building the {@link TableWrite} and {@link TableCommit}. + * + * <p>Example of distributed batch writing: + * + * <pre>{@code + * // 1. Create a WriteBuilder (Serializable) + * Table table = catalog.getTable(...); + * WriteBuilder builder = table.newWriteBuilder(); + * + * // 2. Write records in distributed tasks + * TableWrite write = builder.newWrite(); + * write.write(...); + * write.write(...); + * write.write(...); + * List<CommitMessage> messages = write.prepareCommit(true, 0); + * + * // 3. Collect all CommitMessages to a global node + * + * // 3.1 commit + * TableCommit commit = builder.newCommit(); + * if (commit.filterCommitted(0)) { + * // It has already been committed. In order to avoid + * // repeated committing, return directly. + * return; + * } + * commit.commit(0, allCommitMessages()); + * + * // 3.2. Expire snapshots and partitions + * commit.expireSnapshots(); + * commit.expirePartitions(); + * + * }</pre> + * + * @since 0.4.0 + */ +@Experimental +public interface WriteBuilder extends Serializable { + + /** A name to identify this table. */ + String tableName(); + + /** Returns the row type of this table. */ + RowType rowType(); + + /** Get commit user, set by {@link #withCommitUser}. */ + String commitUser(); + + /** + * Set commit user, the default value is UUID. The commit user used by {@link TableWrite} and + * {@link TableCommit} must be the same, otherwise there will be some conflicts. + */ + WriteBuilder withCommitUser(String commitUser); Review Comment: > Set commit user, the default value is UUID. the default value is a random UUID. > he commit user used by TableWrite and TableCommit must be the same, otherwise there will be some conflicts. Users cannot set commitUser separately for `TableWrite` and `TableCommit` with public API. No need for this statement. ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ + default boolean filterCommitted(long commitIdentifier) { Review Comment: What does the return value mean? If it returns true, should user commit this snapshot again or not? ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ + default boolean filterCommitted(long commitIdentifier) { + Set<Long> filter = filterCommitted(Collections.singleton(commitIdentifier)); + return filter.contains(commitIdentifier); } - public List<ManifestCommittable> filterCommitted(List<ManifestCommittable> committables) { - return commit.filterCommitted(committables); - } - - public void commit(long identifier, List<FileCommittable> fileCommittables) { - ManifestCommittable committable = new ManifestCommittable(identifier); - for (FileCommittable fileCommittable : fileCommittables) { - committable.addFileCommittable(fileCommittable); - } - commit(Collections.singletonList(committable)); - } - - public void commit(List<ManifestCommittable> committables) { - if (overwritePartitions == null) { - for (ManifestCommittable committable : committables) { - commit.commit(committable, new HashMap<>()); - } - } else { - ManifestCommittable committable; - if (committables.size() > 1) { - throw new RuntimeException( - "Multiple committables appear in overwrite mode, this may be a bug, please report it: " - + committables); - } else if (committables.size() == 1) { - committable = committables.get(0); - } else { - // create an empty committable - // identifier is Long.MAX_VALUE, come from batch job - // TODO maybe it can be produced by CommitterOperator - committable = new ManifestCommittable(Long.MAX_VALUE); - } - commit.overwrite(overwritePartitions, committable, new HashMap<>()); - } - - if (expire != null) { - expire.expire(); - } - - if (partitionExpire != null) { - partitionExpire.expire(); - } - } - - @Override - public void close() throws Exception { - if (lock != null) { - lock.close(); - } - } + /** Filter committed commits. This method is used for failover cases. */ + Set<Long> filterCommitted(Set<Long> commitIdentifiers); + + /** + * Create a new commit. One commit may generate two snapshots, one for adding new files and the + * other for compaction. + */ + void commit(long commitIdentifier, List<CommitMessage> commitMessages); Review Comment: Explain what is a `commitIdentifier` and stress that it must become larger for later commits. ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ + default boolean filterCommitted(long commitIdentifier) { + Set<Long> filter = filterCommitted(Collections.singleton(commitIdentifier)); + return filter.contains(commitIdentifier); } - public List<ManifestCommittable> filterCommitted(List<ManifestCommittable> committables) { - return commit.filterCommitted(committables); - } - - public void commit(long identifier, List<FileCommittable> fileCommittables) { - ManifestCommittable committable = new ManifestCommittable(identifier); - for (FileCommittable fileCommittable : fileCommittables) { - committable.addFileCommittable(fileCommittable); - } - commit(Collections.singletonList(committable)); - } - - public void commit(List<ManifestCommittable> committables) { - if (overwritePartitions == null) { - for (ManifestCommittable committable : committables) { - commit.commit(committable, new HashMap<>()); - } - } else { - ManifestCommittable committable; - if (committables.size() > 1) { - throw new RuntimeException( - "Multiple committables appear in overwrite mode, this may be a bug, please report it: " - + committables); - } else if (committables.size() == 1) { - committable = committables.get(0); - } else { - // create an empty committable - // identifier is Long.MAX_VALUE, come from batch job - // TODO maybe it can be produced by CommitterOperator - committable = new ManifestCommittable(Long.MAX_VALUE); - } - commit.overwrite(overwritePartitions, committable, new HashMap<>()); - } - - if (expire != null) { - expire.expire(); - } - - if (partitionExpire != null) { - partitionExpire.expire(); - } - } - - @Override - public void close() throws Exception { - if (lock != null) { - lock.close(); - } - } + /** Filter committed commits. This method is used for failover cases. */ + Set<Long> filterCommitted(Set<Long> commitIdentifiers); + + /** + * Create a new commit. One commit may generate two snapshots, one for adding new files and the + * other for compaction. + */ + void commit(long commitIdentifier, List<CommitMessage> commitMessages); + + /** + * Expire snapshots according to three options: + * + * <ol> + * <li>'snapshot.time-retained': The maximum time of completed snapshots to retain. + * <li>'snapshot.num-retained.min': The minimum number of completed snapshots to retain. + * <li>'snapshot.num-retained.max': The maximum number of completed snapshots to retain. + * </ol> + * + * <p>Even if an exception occurs during the process, you can continue to delete files later, + * and there will be no zombie files left. + */ + void expireSnapshots(); Review Comment: I was expecting a separate `TableExpire` class with a `void expireSnapshots(int numRetained)` method. It is really weird to control the behavior through an option map when you could have done it by a mere Java function parameter. If you insist to use an option map, it might be better to still merge commit and expire together. However the comments of that method must be clear, so user knows that some snapshot may be cleaned up after the commit. ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCommit.java: ########## @@ -18,121 +18,76 @@ package org.apache.flink.table.store.table.sink; -import org.apache.flink.table.store.file.manifest.ManifestCommittable; -import org.apache.flink.table.store.file.operation.FileStoreCommit; -import org.apache.flink.table.store.file.operation.FileStoreExpire; -import org.apache.flink.table.store.file.operation.Lock; -import org.apache.flink.table.store.file.operation.PartitionExpire; - -import javax.annotation.Nullable; +import org.apache.flink.table.store.annotation.Experimental; +import org.apache.flink.table.store.table.Table; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Set; /** - * An abstraction layer above {@link FileStoreCommit} and {@link FileStoreExpire} to provide - * snapshot commit and expiration. + * Create and commit snapshots of a {@link Table}. Snapshots are produced from {@link + * CommitMessage}s, which themselves are generated by {@link TableWrite}. Also see {@link + * TableWrite#prepareCommit}. + * + * <ol> + * <li>Before calling {@link TableCommit#commit}, if user cannot determine if this commit is done + * before, user should first call {@link TableCommit#filterCommitted}. + * <li>Commit may fail and throw an exception. Before committing, it will first check for + * conflicts by checking if all files to be removed currently exists. + * </ol> + * + * @since 0.4.0 */ -public class TableCommit implements AutoCloseable { - - private final FileStoreCommit commit; - @Nullable private final FileStoreExpire expire; - @Nullable private final PartitionExpire partitionExpire; - - @Nullable private List<Map<String, String>> overwritePartitions = null; - @Nullable private Lock lock; - - public TableCommit( - FileStoreCommit commit, - @Nullable FileStoreExpire expire, - @Nullable PartitionExpire partitionExpire) { - this.commit = commit; - this.expire = expire; - this.partitionExpire = partitionExpire; - } - - public TableCommit withOverwritePartition(@Nullable Map<String, String> overwritePartition) { - if (overwritePartition != null) { - this.overwritePartitions = Collections.singletonList(overwritePartition); - } - return this; - } - - public TableCommit withOverwritePartitions( - @Nullable List<Map<String, String>> overwritePartitions) { - this.overwritePartitions = overwritePartitions; - return this; - } - - public TableCommit withLock(Lock lock) { - commit.withLock(lock); - - if (expire != null) { - expire.withLock(lock); - } - - if (partitionExpire != null) { - partitionExpire.withLock(lock); - } - - this.lock = lock; - return this; - } - - public TableCommit withCreateEmptyCommit(boolean createEmptyCommit) { - commit.withCreateEmptyCommit(createEmptyCommit); - return this; +@Experimental +public interface TableCommit extends AutoCloseable { + + /** + * Default ignore empty commit, if this is set to false, when there is no new data, an empty + * commit will also be created. + * + * <p>NOTE: It is recommended to set 'ignoreEmptyCommit' to false in streaming write, in order + * to better remove duplicate commits (See {@link #filterCommitted}). + */ + TableCommit ignoreEmptyCommit(boolean ignoreEmptyCommit); + + /** Filter committed commit. This method is used for failover cases. */ + default boolean filterCommitted(long commitIdentifier) { + Set<Long> filter = filterCommitted(Collections.singleton(commitIdentifier)); + return filter.contains(commitIdentifier); } - public List<ManifestCommittable> filterCommitted(List<ManifestCommittable> committables) { - return commit.filterCommitted(committables); - } - - public void commit(long identifier, List<FileCommittable> fileCommittables) { - ManifestCommittable committable = new ManifestCommittable(identifier); - for (FileCommittable fileCommittable : fileCommittables) { - committable.addFileCommittable(fileCommittable); - } - commit(Collections.singletonList(committable)); - } - - public void commit(List<ManifestCommittable> committables) { - if (overwritePartitions == null) { - for (ManifestCommittable committable : committables) { - commit.commit(committable, new HashMap<>()); - } - } else { - ManifestCommittable committable; - if (committables.size() > 1) { - throw new RuntimeException( - "Multiple committables appear in overwrite mode, this may be a bug, please report it: " - + committables); - } else if (committables.size() == 1) { - committable = committables.get(0); - } else { - // create an empty committable - // identifier is Long.MAX_VALUE, come from batch job - // TODO maybe it can be produced by CommitterOperator - committable = new ManifestCommittable(Long.MAX_VALUE); - } - commit.overwrite(overwritePartitions, committable, new HashMap<>()); - } - - if (expire != null) { - expire.expire(); - } - - if (partitionExpire != null) { - partitionExpire.expire(); - } - } - - @Override - public void close() throws Exception { - if (lock != null) { - lock.close(); - } - } + /** Filter committed commits. This method is used for failover cases. */ + Set<Long> filterCommitted(Set<Long> commitIdentifiers); Review Comment: Same problem. What does the return value mean? Shall users commit all snapshots in the returned set again? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org