You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tephra.apache.org by anew <gi...@git.apache.org> on 2017/08/17 23:28:09 UTC

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

GitHub user anew opened a pull request:

    https://github.com/apache/incubator-tephra/pull/47

    [TEPHRA-240] Include conflicting key and client id in TransactionConflictException

    - Enhances TransactionConflictException to have fields for the transaction id, the conflicting change, and the conflicting client id
    - TransactionManager to retain the client id for change sets after the transaction has committed
    - A configuration for whether and how long it should be retained. This is to allow disabling this feature (and it effect on memory)
    - TransactionManager.(can)Commit now throws TrandactionConflictException instead of returning a boolean
    - new thrift methods (and a new thrift exception). The existing thrift methods will remain backward-compatible
    - change TransactionSystemClient and TransactionContext to use the new methods
    - new test case (and existing test case changes) for this
    - a few unrelated warnings have been removed from TransactionManager
    - a bit of test case refactoring to remove duplication of code

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/anew/incubator-tephra tephra-240

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-tephra/pull/47.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #47
    
----
commit 205714fdd126ab7537f9f66e6e156c9736f54646
Author: anew <an...@apache.org>
Date:   2017-08-17T23:21:52Z

    [TEPHRA-240] Include conflicting key and client id in TransactionConflictException

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138437903
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionManager.java ---
    @@ -853,46 +867,45 @@ private void advanceWritePointer(long writePointer) {
         }
       }
     
    -  public boolean canCommit(Transaction tx, Collection<byte[]> changeIds)
    -    throws TransactionNotInProgressException, TransactionSizeException {
    +  public void canCommit(long txId, Collection<byte[]> changeIds)
    +    throws TransactionNotInProgressException, TransactionSizeException, TransactionConflictException {
     
         txMetricsCollector.rate("canCommit");
         Stopwatch timer = new Stopwatch().start();
    -    InProgressTx inProgressTx = inProgress.get(tx.getTransactionId());
    +    InProgressTx inProgressTx = inProgress.get(txId);
         if (inProgressTx == null) {
           synchronized (this) {
             // invalid transaction, either this has timed out and moved to invalid, or something else is wrong.
    -        if (invalidTxList.contains(tx.getTransactionId())) {
    +        if (invalidTxList.contains(txId)) {
               throw new TransactionNotInProgressException(
                 String.format(
    -              "canCommit() is called for transaction %d that is not in progress (it is known to be invalid)",
    -              tx.getTransactionId()));
    +              "canCommit() is called for transaction %d that is not in progress (it is known to be invalid)", txId));
             } else {
               throw new TransactionNotInProgressException(
    -            String.format("canCommit() is called for transaction %d that is not in progress", tx.getTransactionId()));
    +            String.format("canCommit() is called for transaction %d that is not in progress", txId));
             }
           }
         }
     
         Set<ChangeId> set =
    -      validateChangeSet(tx, changeIds, inProgressTx.clientId != null ? inProgressTx.clientId : DEFAULT_CLIENTID);
    -
    -    if (hasConflicts(tx, set)) {
    -      return false;
    +      validateChangeSet(txId, changeIds, inProgressTx.clientId != null ? inProgressTx.clientId : DEFAULT_CLIENTID);
    +    for (byte[] change : changeIds) {
    --- End diff --
    
    Good catch. I forgot to remove this from existing code. 


---

[GitHub] incubator-tephra issue #47: [TEPHRA-240] Include conflicting key and client ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on the issue:

    https://github.com/apache/incubator-tephra/pull/47
  
    squashed and rebased on latest master.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138437669
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionSystemClient.java ---
    @@ -89,21 +89,38 @@
        *
        * @param tx transaction to verify
        * @param changeIds ids of changes made by transaction
    -   * @return true if transaction can be committed otherwise false
    -   * @throws TransactionSizeException if the size of the chgange set exceeds the allowed limit
    -   * @throws TransactionNotInProgressException if the transaction is not in progress; most likely it has timed out.
    +   *
    +   * @throws TransactionSizeException if the size of the change set exceeds the allowed limit
    +   * @throws TransactionConflictException if the change set has a conflict with an overlapping transaction
    +   * @throws TransactionNotInProgressException if the transaction is not in progress; most likely it has timed out
        */
    -  boolean canCommitOrThrow(Transaction tx, Collection<byte[]> changeIds) throws TransactionFailureException;
    +  void canCommitOrThrow(Transaction tx, Collection<byte[]> changeIds)
    +    throws TransactionNotInProgressException, TransactionConflictException, TransactionSizeException;
    --- End diff --
    
    Replaced for canCommitOrThrow() and commitOrThrow(). 


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138490048
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionManager.java ---
    @@ -206,12 +210,19 @@ public TransactionManager(Configuration conf, @Nonnull TransactionStateStorage p
         // TODO: REMOVE WITH txnBackwardsCompatCheck()
         longTimeoutTolerance = conf.getLong("data.tx.long.timeout.tolerance", 10000);
     
    -    //
    +    ClientIdRetention retention = ClientIdRetention.valueOf(
    --- End diff --
    
    Hmmm. We don't do that for the other configurations... if any of the numeric properties cannot be parsed as a number, it also fails. I think it is a good idea to fail on invalid configuration, because if there is a configuration that is present, then that is most likely with the intention to change the default. If there is a typo or some other error, and we only log a warning, that warning is likely to go unnoticed and the system will behave in a way that was intended, and that will go undetected until it causes failures. 


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138435693
  
    --- Diff: tephra-api/src/main/java/org/apache/tephra/TransactionConflictException.java ---
    @@ -22,11 +22,50 @@
      * Thrown to indicate transaction conflict occurred when trying to commit a transaction.
      */
     public class TransactionConflictException extends TransactionFailureException {
    +
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message) {
         super(message);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
       }
     
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message, Throwable cause) {
         super(message, cause);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
    +  }
    +
    +  public TransactionConflictException(long transactionId, String conflictingChange, String conflictingClient) {
    +    super(String.format("Transaction %d conflicts with %s on change key '%s'", transactionId,
    +                        conflictingClient == null ? "unknown client" : conflictingClient, conflictingChange));
    +    this.transactionId = transactionId;
    +    this.conflictingChange = conflictingChange;
    +    this.conflictingClient = conflictingClient;
    +  }
    +
    +  private final Long transactionId;
    --- End diff --
    
    ehm, yes, must be by accident


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138239467
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionSystemClient.java ---
    @@ -89,21 +89,38 @@
        *
        * @param tx transaction to verify
        * @param changeIds ids of changes made by transaction
    -   * @return true if transaction can be committed otherwise false
    -   * @throws TransactionSizeException if the size of the chgange set exceeds the allowed limit
    -   * @throws TransactionNotInProgressException if the transaction is not in progress; most likely it has timed out.
    +   *
    +   * @throws TransactionSizeException if the size of the change set exceeds the allowed limit
    +   * @throws TransactionConflictException if the change set has a conflict with an overlapping transaction
    +   * @throws TransactionNotInProgressException if the transaction is not in progress; most likely it has timed out
        */
    -  boolean canCommitOrThrow(Transaction tx, Collection<byte[]> changeIds) throws TransactionFailureException;
    +  void canCommitOrThrow(Transaction tx, Collection<byte[]> changeIds)
    +    throws TransactionNotInProgressException, TransactionConflictException, TransactionSizeException;
    --- End diff --
    
    Would declaring `TransactionFailureException` here help in making the API resilient to some future changes - in-case we need to add new exceptions to indicate a transaction failure?


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138238770
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionContext.java ---
    @@ -311,25 +304,16 @@ private void persist() throws TransactionFailureException {
       }
     
       private void commit() throws TransactionFailureException {
    -    boolean commitSuccess = false;
         try {
    -      commitSuccess = txClient.commit(currentTx);
    -    } catch (TransactionNotInProgressException e) {
    -      String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId());
    -      LOG.warn(message, e);
    -      abort(new TransactionFailureException(message, e));
    -      // abort will throw that exception
    +      txClient.commitOrThrow(currentTx);
    +    } catch (TransactionNotInProgressException | TransactionConflictException e) {
    --- End diff --
    
    This should also catch `TransactionFailureException` like the catch block of method `checkForConflicts()` above, right?


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138436222
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionContext.java ---
    @@ -311,25 +304,16 @@ private void persist() throws TransactionFailureException {
       }
     
       private void commit() throws TransactionFailureException {
    -    boolean commitSuccess = false;
         try {
    -      commitSuccess = txClient.commit(currentTx);
    -    } catch (TransactionNotInProgressException e) {
    -      String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId());
    -      LOG.warn(message, e);
    -      abort(new TransactionFailureException(message, e));
    -      // abort will throw that exception
    +      txClient.commitOrThrow(currentTx);
    +    } catch (TransactionNotInProgressException | TransactionConflictException e) {
    --- End diff --
    
    it does not make a difference, but may be more future-proof.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138480498
  
    --- Diff: tephra-core/src/main/thrift/transaction.thrift ---
    @@ -73,15 +79,21 @@ service TTransactionServer {
       // TODO remove this as it was replaced with startShortWithTimeout in 0.10
       TTransaction startShortTimeout(1: i32 timeout),
       TTransaction startShortClientId(1: string clientId) throws (1: TGenericException e),
    -  TTransaction startShortWithClientIdAndTimeOut(1: string clientId, 2: i32 timeout) throws (1:TGenericException e),
    -  TTransaction startShortWithTimeout(1: i32 timeout) throws (1:TGenericException e),
    -  TBoolean canCommitTx(1: TTransaction tx, 2: set<binary> changes) throws (1:TTransactionNotInProgressException e),
    -  TBoolean canCommitOrThrow(1: TTransaction tx, 2: set<binary> changes) throws (1:TTransactionNotInProgressException e,
    -                                                                                2:TGenericException g,),
    +  TTransaction startShortWithClientIdAndTimeOut(1: string clientId, 2: i32 timeout) throws (1: TGenericException e),
    +  TTransaction startShortWithTimeout(1: i32 timeout) throws (1: TGenericException e),
    +  // TODO remove this as it was replaced with canCommitOrThrow in 0.13
    +  TBoolean canCommitTx(1: TTransaction tx, 2: set<binary> changes) throws (1: TTransactionNotInProgressException e),
    +  void canCommitOrThrow(1: i64 tx, 2: set<binary> changes) throws (1: TTransactionNotInProgressException e,
    +                                                                   2: TTransactionConflictException c,
    +                                                                   3: TGenericException g),
    +  // TODO remove this as it was replaced with commitWithExn in 0.13
       TBoolean commitTx(1: TTransaction tx) throws (1:TTransactionNotInProgressException e),
    +  void commitOrThrow(1: i64 txId, 2: i64 wp) throws (1: TTransactionNotInProgressException e,
    --- End diff --
    
    I missed it yesterday, it would be good to use TransactionFailureException here too for future-proofing.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138477861
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionManager.java ---
    @@ -206,12 +210,19 @@ public TransactionManager(Configuration conf, @Nonnull TransactionStateStorage p
         // TODO: REMOVE WITH txnBackwardsCompatCheck()
         longTimeoutTolerance = conf.getLong("data.tx.long.timeout.tolerance", 10000);
     
    -    //
    +    ClientIdRetention retention = ClientIdRetention.valueOf(
    --- End diff --
    
    It would be good to catch the exception during enum conversion and use the default value. This way transaction manager will still startup on a misconfiguration.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138237949
  
    --- Diff: tephra-api/src/main/java/org/apache/tephra/TransactionConflictException.java ---
    @@ -22,11 +22,50 @@
      * Thrown to indicate transaction conflict occurred when trying to commit a transaction.
      */
     public class TransactionConflictException extends TransactionFailureException {
    +
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message) {
         super(message);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
       }
     
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message, Throwable cause) {
         super(message, cause);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
    +  }
    +
    +  public TransactionConflictException(long transactionId, String conflictingChange, String conflictingClient) {
    +    super(String.format("Transaction %d conflicts with %s on change key '%s'", transactionId,
    +                        conflictingClient == null ? "unknown client" : conflictingClient, conflictingChange));
    +    this.transactionId = transactionId;
    +    this.conflictingChange = conflictingChange;
    +    this.conflictingClient = conflictingClient;
    +  }
    +
    +  private final Long transactionId;
    --- End diff --
    
    It would be good if the fields are defined before the constructor definitions.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138435657
  
    --- Diff: tephra-api/src/main/java/org/apache/tephra/TransactionConflictException.java ---
    @@ -22,11 +22,50 @@
      * Thrown to indicate transaction conflict occurred when trying to commit a transaction.
      */
     public class TransactionConflictException extends TransactionFailureException {
    +
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message) {
         super(message);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
       }
     
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message, Throwable cause) {
         super(message, cause);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
    +  }
    +
    +  public TransactionConflictException(long transactionId, String conflictingChange, String conflictingClient) {
    --- End diff --
    
    ok


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138270921
  
    --- Diff: tephra-core/src/main/java/org/apache/tephra/TransactionManager.java ---
    @@ -853,46 +867,45 @@ private void advanceWritePointer(long writePointer) {
         }
       }
     
    -  public boolean canCommit(Transaction tx, Collection<byte[]> changeIds)
    -    throws TransactionNotInProgressException, TransactionSizeException {
    +  public void canCommit(long txId, Collection<byte[]> changeIds)
    +    throws TransactionNotInProgressException, TransactionSizeException, TransactionConflictException {
     
         txMetricsCollector.rate("canCommit");
         Stopwatch timer = new Stopwatch().start();
    -    InProgressTx inProgressTx = inProgress.get(tx.getTransactionId());
    +    InProgressTx inProgressTx = inProgress.get(txId);
         if (inProgressTx == null) {
           synchronized (this) {
             // invalid transaction, either this has timed out and moved to invalid, or something else is wrong.
    -        if (invalidTxList.contains(tx.getTransactionId())) {
    +        if (invalidTxList.contains(txId)) {
               throw new TransactionNotInProgressException(
                 String.format(
    -              "canCommit() is called for transaction %d that is not in progress (it is known to be invalid)",
    -              tx.getTransactionId()));
    +              "canCommit() is called for transaction %d that is not in progress (it is known to be invalid)", txId));
             } else {
               throw new TransactionNotInProgressException(
    -            String.format("canCommit() is called for transaction %d that is not in progress", tx.getTransactionId()));
    +            String.format("canCommit() is called for transaction %d that is not in progress", txId));
             }
           }
         }
     
         Set<ChangeId> set =
    -      validateChangeSet(tx, changeIds, inProgressTx.clientId != null ? inProgressTx.clientId : DEFAULT_CLIENTID);
    -
    -    if (hasConflicts(tx, set)) {
    -      return false;
    +      validateChangeSet(txId, changeIds, inProgressTx.clientId != null ? inProgressTx.clientId : DEFAULT_CLIENTID);
    +    for (byte[] change : changeIds) {
    --- End diff --
    
    I don't think this for-loop is needed as method `validateChangeSet()` already returns `Set<ChangeId>`.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-tephra/pull/47


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by poornachandra <gi...@git.apache.org>.
Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138222710
  
    --- Diff: tephra-api/src/main/java/org/apache/tephra/TransactionConflictException.java ---
    @@ -22,11 +22,50 @@
      * Thrown to indicate transaction conflict occurred when trying to commit a transaction.
      */
     public class TransactionConflictException extends TransactionFailureException {
    +
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message) {
         super(message);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
       }
     
    +  /**
    +   * @deprecated since 0.13-incubating. Use {@link #TransactionConflictException(long, String, String)} instead.
    +   */
    +  @Deprecated
       public TransactionConflictException(String message, Throwable cause) {
         super(message, cause);
    +    transactionId = null;
    +    conflictingChange = null;
    +    conflictingClient = null;
    +  }
    +
    +  public TransactionConflictException(long transactionId, String conflictingChange, String conflictingClient) {
    --- End diff --
    
    I think `conflictingKey` would be a more appropriate name for `conflictingChange`.


---

[GitHub] incubator-tephra pull request #47: [TEPHRA-240] Include conflicting key and ...

Posted by anew <gi...@git.apache.org>.
Github user anew commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/47#discussion_r138489380
  
    --- Diff: tephra-core/src/main/thrift/transaction.thrift ---
    @@ -73,15 +79,21 @@ service TTransactionServer {
       // TODO remove this as it was replaced with startShortWithTimeout in 0.10
       TTransaction startShortTimeout(1: i32 timeout),
       TTransaction startShortClientId(1: string clientId) throws (1: TGenericException e),
    -  TTransaction startShortWithClientIdAndTimeOut(1: string clientId, 2: i32 timeout) throws (1:TGenericException e),
    -  TTransaction startShortWithTimeout(1: i32 timeout) throws (1:TGenericException e),
    -  TBoolean canCommitTx(1: TTransaction tx, 2: set<binary> changes) throws (1:TTransactionNotInProgressException e),
    -  TBoolean canCommitOrThrow(1: TTransaction tx, 2: set<binary> changes) throws (1:TTransactionNotInProgressException e,
    -                                                                                2:TGenericException g,),
    +  TTransaction startShortWithClientIdAndTimeOut(1: string clientId, 2: i32 timeout) throws (1: TGenericException e),
    +  TTransaction startShortWithTimeout(1: i32 timeout) throws (1: TGenericException e),
    +  // TODO remove this as it was replaced with canCommitOrThrow in 0.13
    +  TBoolean canCommitTx(1: TTransaction tx, 2: set<binary> changes) throws (1: TTransactionNotInProgressException e),
    +  void canCommitOrThrow(1: i64 tx, 2: set<binary> changes) throws (1: TTransactionNotInProgressException e,
    +                                                                   2: TTransactionConflictException c,
    +                                                                   3: TGenericException g),
    +  // TODO remove this as it was replaced with commitWithExn in 0.13
       TBoolean commitTx(1: TTransaction tx) throws (1:TTransactionNotInProgressException e),
    +  void commitOrThrow(1: i64 txId, 2: i64 wp) throws (1: TTransactionNotInProgressException e,
    --- End diff --
    
    Can't do that. Thrift does not support exception hierarchies. I did add the TGenericException such that we can transport other exception types in the future.


---