You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/04 18:22:14 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #1849: API for supporting a TransactionalCatalog to enable multi-table transactions

yyanyy commented on a change in pull request #1849:
URL: https://github.com/apache/iceberg/pull/1849#discussion_r535890952



##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is

Review comment:
       Is this conflicting writes concept talking about the same thing as referred in the "Commit Conflict Resolution and Retry" of the [spec](https://iceberg.apache.org/spec/) since each table will have to detect and resolve its own conflict individually? I guess the only difference is that for multi table transactions, if all tables don't have conflict except for one that couldn't resolve, all tables will be rolled back? 

##########
File path: api/src/main/java/org/apache/iceberg/Table.java
##########
@@ -41,6 +41,13 @@ default String name() {
 
   /**
    * Refresh the current table metadata.
+   *
+   * <p>If this table is associated with a TransactionalCatalog, this refresh will be bounded by
+   * the visibility that the {@code IsolationLevel} of that transaction exposes. For example, if
+   * we are in a context of {@code READ_COMMITTED}, this refresh will update to the latest state
+   * of the table. However, in the case of {@code SERIALIZABLE} where this table hasn't mutated
+   * within this transaction, calling refresh will have no impact as the isolation level

Review comment:
       "this table hasn't mutated within this transaction" may sound like implying that if this transaction contains table mutation changes, `refresh` may have impact, which I think is not true? I guess what you were saying was if other transactions committed to this table successfully when this transaction is half way through, refresh in this transaction will still return the same state as when this transaction begins. 

##########
File path: api/src/main/java/org/apache/iceberg/catalog/TransactionalCatalog.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.iceberg.catalog;
+
+import org.apache.iceberg.catalog.SupportsCatalogTransactions.IsolationLevel;
+import org.apache.iceberg.catalog.SupportsCatalogTransactions.LockingMode;
+import org.apache.iceberg.exceptions.CommitFailedException;
+
+/**
+ * A {@link Catalog} that applies all mutations within a single transaction.
+ *
+ * <p>A TransactionalCatalog can spawn child transactions for multiple operations on different
+ * tables. All operations will be done within the context of a single Catalog-level transaction
+ * and they will either all be successful or all fail.
+ *
+ * <p>A TransactionalCatalog is initially active upon creation and will remain so until one of
+ * the following terminal actions occurs:
+ * <ul>
+ * <li>{@link rollback} is called.
+ * <li>{@link commit} is called.
+ * <li>The transaction expires while using Pessimistic {@link LockingMode}.
+ * <li>The transaction is terminated externally (for example, when a locking arbitrator
+ *     determines a deadlock between two transactions has occurred).
+ * <li>The underlying implementation determines that the transaction can no longer complete
+ *     successfully.
+ * </ul>
+ *
+ * <p>When one of the items above occurs, the transaction is no longer valid. Further use
+ * of the transaction will result in a {@link IllegalStateException} being thrown.
+ *
+ * <p>Nested transactions such as creating a new table may fail. Those failures alone do
+ * not necessarily result in a failure of the catalog-level transaction.
+ *
+ * <p>Implementations of {@code TransactionalCatalog} are responsible for monitoring all
+ * table level operations that are spawned from this catalog and ensure that all nested
+ * transactions that are completed successfully are either exposed atomically or not.
+ *
+ */
+public interface TransactionalCatalog extends Catalog, AutoCloseable {
+
+  /**
+   * An internal identifier associated with this transaction.
+   * @return An internal identifier.
+   */
+  String transactionId();
+
+  /**
+   * Return the current {@code IsolationLevel} for this transaction.
+   * @return The IsolationLevel for this transaction.
+   */
+  IsolationLevel isolationLevel();
+
+  /**
+   * Return the {@link LockingMode} for this transaction.
+   * @return The LockingMode for this transaction.
+   */
+  LockingMode lockingMode();
+
+  /**
+   * Whether the current transaction is still active/open.
+   * @return True until a terminal action occurs.
+   */
+  boolean active();
+
+  /**
+   * Aborts the set of operations here and makes this TransactionalCatalog inoperable.
+   *
+   * <p>Once called, no further operations can be done against this catalog. If any
+   * operations are attempted, {@link IllegalStateException} will be thrown.
+   */
+  void rollback();
+
+  /**
+   * Commit the pending changes from all nested transactions against the Catalog.
+   *
+   * <p>Once called, no further operations can be done against this catalog. If any
+   * operations are attempted, {@link IllegalStateException} will be thrown.
+   *
+   * @throws CommitFailedException If the updates cannot be committed due to conflicts.
+   */
+  void commit();
+
+  /**
+   * Close out all resources associated with a transaction.
+   *
+   * <p>This will do a conditional rollback if neither {@code commit} nor {@code rollback}
+   * were called. Standard usage looks like:
+   * <pre>
+   * try(TransactionalCatalog tx = catalog.createTransaction(IsolationLevel.READ_COMMITTED)) {
+   *  doOp1(tx);
+   *  doOp2(tx);
+   *  tx.commit();
+   * }
+   * </pre>
+   * This pattern is designed such that if {@code doOp1()} or {@code doOp2()} throw an exception,
+   * the transaction will be automatically rolled back. If both operations complete successfully,
+   * the close will only close any remaining open resources associated with the transaction.
+   */
+  @Override
+  default void close() {
+    commit();
+    close();

Review comment:
       Wouldn't this cause stackoverflow?

##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is
+   * referenced. Conflicting writes are two mutations to the same object that happen concurrently.
+   * Depending on the particular implementation, the coarseness of this conflict may vary. The
+   * most coarse conflict is any two mutations to the same table. However, some implementations
+   * may consider some of these "absolute" conflicts as allowable by using finer-grained conflict
+   * resolution. For example, two different operations that both append new files to a table may
+   * be in "absolute" conflict but could be resolved automatically as a "safe conflict" by using
+   * a set of automatic implementation-defined conflict resolution rules.
+   */
+  enum IsolationLevel {
+
+    /**
+     * Reading the same table multiple times may result in different versions read of the same
+     * table. A commit can be completed as long as any tables changed externally do not conflict
+     * with any writes within this transaction.
+     */
+    READ_COMMITTED,
+
+    /**
+     * Reading the same table multiple times will result in the same view of that table.
+     * Different tables may come from different snapshots. A commit can be completed as
+     * long as any tables changed externally do not conflict with any writes within this
+     * transaction.
+     */
+    REPEATED_READ,
+
+    /**
+     * A commit will only succeed if there have been no meaningful changes to data read during
+     * the course of this transaction prior to commit. This imposes stricter read guarantees than
+     * {@code REPEATED_READ} (consistent reads per table) as it requires that the reads are
+     * consistent for all tables to a single point in time (or single snapshot of the database).
+     * Additionally, it implies additional requirements around the successful completion of a
+     * write. In order for a write to complete, any entities read during this transaction are also
+     * blocked from changing (via another transaction) post-read in ways that would influence the

Review comment:
       if it's optimistic locking, would it still block other transactions? 

##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is
+   * referenced. Conflicting writes are two mutations to the same object that happen concurrently.
+   * Depending on the particular implementation, the coarseness of this conflict may vary. The
+   * most coarse conflict is any two mutations to the same table. However, some implementations
+   * may consider some of these "absolute" conflicts as allowable by using finer-grained conflict
+   * resolution. For example, two different operations that both append new files to a table may
+   * be in "absolute" conflict but could be resolved automatically as a "safe conflict" by using
+   * a set of automatic implementation-defined conflict resolution rules.
+   */
+  enum IsolationLevel {
+
+    /**
+     * Reading the same table multiple times may result in different versions read of the same
+     * table. A commit can be completed as long as any tables changed externally do not conflict
+     * with any writes within this transaction.
+     */
+    READ_COMMITTED,
+
+    /**
+     * Reading the same table multiple times will result in the same view of that table.
+     * Different tables may come from different snapshots. A commit can be completed as
+     * long as any tables changed externally do not conflict with any writes within this
+     * transaction.
+     */
+    REPEATED_READ,
+
+    /**
+     * A commit will only succeed if there have been no meaningful changes to data read during
+     * the course of this transaction prior to commit. This imposes stricter read guarantees than
+     * {@code REPEATED_READ} (consistent reads per table) as it requires that the reads are
+     * consistent for all tables to a single point in time (or single snapshot of the database).
+     * Additionally, it implies additional requirements around the successful completion of a
+     * write. In order for a write to complete, any entities read during this transaction are also
+     * blocked from changing (via another transaction) post-read in ways that would influence the
+     * writes of this operation. This is also sometimes called snapshot isolation.

Review comment:
       From [wikipedia definition](https://en.wikipedia.org/wiki/Snapshot_isolation) looks like snapshot isolation wouldn't avoid write skew problem while serializable would, so I think we probably don't want to call them interchangeably? 

##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is
+   * referenced. Conflicting writes are two mutations to the same object that happen concurrently.
+   * Depending on the particular implementation, the coarseness of this conflict may vary. The
+   * most coarse conflict is any two mutations to the same table. However, some implementations
+   * may consider some of these "absolute" conflicts as allowable by using finer-grained conflict
+   * resolution. For example, two different operations that both append new files to a table may
+   * be in "absolute" conflict but could be resolved automatically as a "safe conflict" by using
+   * a set of automatic implementation-defined conflict resolution rules.
+   */
+  enum IsolationLevel {
+
+    /**
+     * Reading the same table multiple times may result in different versions read of the same
+     * table. A commit can be completed as long as any tables changed externally do not conflict
+     * with any writes within this transaction.
+     */
+    READ_COMMITTED,
+
+    /**
+     * Reading the same table multiple times will result in the same view of that table.
+     * Different tables may come from different snapshots. A commit can be completed as
+     * long as any tables changed externally do not conflict with any writes within this
+     * transaction.
+     */
+    REPEATED_READ,
+
+    /**
+     * A commit will only succeed if there have been no meaningful changes to data read during
+     * the course of this transaction prior to commit. This imposes stricter read guarantees than

Review comment:
       I think "no meaningful changes to data read" here is a bit vague to understand. For a transaction that get all entries with column `data`=5 from table1 and add them to table2, if there's another transaction that adds `data=5` to table1 that's committed before this transaction finishes, would it be a "meaningful change", but if that transaction adds `data=6` would it still be as it will not change the result of this transaction?

##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is
+   * referenced. Conflicting writes are two mutations to the same object that happen concurrently.
+   * Depending on the particular implementation, the coarseness of this conflict may vary. The
+   * most coarse conflict is any two mutations to the same table. However, some implementations
+   * may consider some of these "absolute" conflicts as allowable by using finer-grained conflict
+   * resolution. For example, two different operations that both append new files to a table may
+   * be in "absolute" conflict but could be resolved automatically as a "safe conflict" by using
+   * a set of automatic implementation-defined conflict resolution rules.
+   */
+  enum IsolationLevel {
+
+    /**
+     * Reading the same table multiple times may result in different versions read of the same
+     * table. A commit can be completed as long as any tables changed externally do not conflict
+     * with any writes within this transaction.
+     */
+    READ_COMMITTED,
+
+    /**
+     * Reading the same table multiple times will result in the same view of that table.
+     * Different tables may come from different snapshots. A commit can be completed as
+     * long as any tables changed externally do not conflict with any writes within this
+     * transaction.
+     */
+    REPEATED_READ,

Review comment:
       nit: I think I normally see "repeatable read" instead of "repeated"? 

##########
File path: api/src/main/java/org/apache/iceberg/catalog/SupportsCatalogTransactions.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.catalog;
+
+import java.util.Set;
+
+/**
+ * Catalog methods for working with catalog-level transactions.
+ *
+ * <p>Catalog implementations are not required to support catalog-level transactional state.
+ * If they do, they may support one or more {@code IsolationLevel}s and one or more
+ * {@code LockingMode}s.
+ */
+public interface SupportsCatalogTransactions {
+
+  /**
+   * The level of isolation for a catalog-level transaction.
+   *
+   * <p>Isolation covers both what data is read and what data can be written.
+   *
+   * <p>At all levels, data is only visible if it is either committed by another transaction or
+   * committed by a nested transaction within this catalog-level transaction.
+   *
+   * <p>Individual nested Table transactions may be "rebased" to expose updated versions of a
+   * table if the isolation level allows that behavior.
+   *
+   * <p>In the definitions of each isolation level, the concept of conflicting writes is
+   * referenced. Conflicting writes are two mutations to the same object that happen concurrently.
+   * Depending on the particular implementation, the coarseness of this conflict may vary. The
+   * most coarse conflict is any two mutations to the same table. However, some implementations
+   * may consider some of these "absolute" conflicts as allowable by using finer-grained conflict
+   * resolution. For example, two different operations that both append new files to a table may
+   * be in "absolute" conflict but could be resolved automatically as a "safe conflict" by using
+   * a set of automatic implementation-defined conflict resolution rules.
+   */
+  enum IsolationLevel {
+
+    /**
+     * Reading the same table multiple times may result in different versions read of the same

Review comment:
       "multiple times within a single transaction"?
   
   




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org