1 package org.apache.lucene.util;
 
   3 import java.io.IOException;
 
   7  * Licensed to the Apache Software Foundation (ASF) under one or more
 
   8  * contributor license agreements.  See the NOTICE file distributed with
 
   9  * this work for additional information regarding copyright ownership.
 
  10  * The ASF licenses this file to You under the Apache License, Version 2.0
 
  11  * (the "License"); you may not use this file except in compliance with
 
  12  * the License.  You may obtain a copy of the License at
 
  14  *     http://www.apache.org/licenses/LICENSE-2.0
 
  16  * Unless required by applicable law or agreed to in writing, software
 
  17  * distributed under the License is distributed on an "AS IS" BASIS,
 
  18  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 
  19  * See the License for the specific language governing permissions and
 
  20  * limitations under the License.
 
  24  * A utility for executing 2-phase commit on several objects.
 
  27  * @lucene.experimental
 
  29 public final class TwoPhaseCommitTool {
 
  32    * A wrapper of a {@link TwoPhaseCommit}, which delegates all calls to the
 
  33    * wrapped object, passing the specified commitData. This object is useful for
 
  34    * use with {@link TwoPhaseCommitTool#execute(TwoPhaseCommit...)} if one would
 
  35    * like to store commitData as part of the commit.
 
  37   public static final class TwoPhaseCommitWrapper implements TwoPhaseCommit {
 
  39     private final TwoPhaseCommit tpc;
 
  40     private  final Map<String, String> commitData;
 
  42     public TwoPhaseCommitWrapper(TwoPhaseCommit tpc, Map<String, String> commitData) {
 
  44       this.commitData = commitData;
 
  47     public void prepareCommit() throws IOException {
 
  48       prepareCommit(commitData);
 
  51     public void prepareCommit(Map<String, String> commitData) throws IOException {
 
  52       tpc.prepareCommit(this.commitData);
 
  55     public void commit() throws IOException {
 
  59     public void commit(Map<String, String> commitData) throws IOException {
 
  60       tpc.commit(this.commitData);
 
  63     public void rollback() throws IOException {
 
  69    * Thrown by {@link TwoPhaseCommitTool#execute(TwoPhaseCommit...)} when an
 
  70    * object fails to prepareCommit().
 
  72   public static class PrepareCommitFailException extends IOException {
 
  74     public PrepareCommitFailException(Throwable cause, TwoPhaseCommit obj) {
 
  75       super("prepareCommit() failed on " + obj);
 
  82    * Thrown by {@link TwoPhaseCommitTool#execute(TwoPhaseCommit...)} when an
 
  83    * object fails to commit().
 
  85   public static class CommitFailException extends IOException {
 
  87     public CommitFailException(Throwable cause, TwoPhaseCommit obj) {
 
  88       super("commit() failed on " + obj);
 
  94   /** rollback all objects, discarding any exceptions that occur. */
 
  95   private static void rollback(TwoPhaseCommit... objects) {
 
  96     for (TwoPhaseCommit tpc : objects) {
 
  97       // ignore any exception that occurs during rollback - we want to ensure
 
  98       // all objects are rolled-back.
 
 102         } catch (Throwable t) {}
 
 108    * Executes a 2-phase commit algorithm by first
 
 109    * {@link TwoPhaseCommit#prepareCommit()} all objects and only if all succeed,
 
 110    * it proceeds with {@link TwoPhaseCommit#commit()}. If any of the objects
 
 111    * fail on either the preparation or actual commit, it terminates and
 
 112    * {@link TwoPhaseCommit#rollback()} all of them.
 
 114    * <b>NOTE:</b> it may happen that an object fails to commit, after few have
 
 115    * already successfully committed. This tool will still issue a rollback
 
 116    * instruction on them as well, but depending on the implementation, it may
 
 117    * not have any effect.
 
 119    * <b>NOTE:</b> if any of the objects are {@code null}, this method simply
 
 122    * @throws PrepareCommitFailException
 
 123    *           if any of the objects fail to
 
 124    *           {@link TwoPhaseCommit#prepareCommit()}
 
 125    * @throws CommitFailException
 
 126    *           if any of the objects fail to {@link TwoPhaseCommit#commit()}
 
 128   public static void execute(TwoPhaseCommit... objects)
 
 129       throws PrepareCommitFailException, CommitFailException {
 
 130     TwoPhaseCommit tpc = null;
 
 132       // first, all should successfully prepareCommit()
 
 133       for (int i = 0; i < objects.length; i++) {
 
 139     } catch (Throwable t) {
 
 140       // first object that fails results in rollback all of them and
 
 141       // throwing an exception.
 
 143       throw new PrepareCommitFailException(t, tpc);
 
 146     // If all successfully prepareCommit(), attempt the actual commit()
 
 148       for (int i = 0; i < objects.length; i++) {
 
 154     } catch (Throwable t) {
 
 155       // first object that fails results in rollback all of them and
 
 156       // throwing an exception.
 
 158       throw new CommitFailException(t, tpc);