001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.master.procedure;
020
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.util.concurrent.CountDownLatch;
024
025import org.apache.yetus.audience.InterfaceAudience;
026import org.apache.yetus.audience.InterfaceStability;
027import org.apache.hadoop.hbase.client.VersionInfoUtil;
028import org.apache.hadoop.hbase.procedure2.Procedure;
029
030/**
031 * Latch used by the Master to have the prepare() sync behaviour for old
032 * clients, that can only get exceptions in a synchronous way.
033 */
034@InterfaceAudience.Private
035@InterfaceStability.Evolving
036public abstract class ProcedurePrepareLatch {
037  private static final NoopLatch noopLatch = new NoopLatch();
038
039  /**
040   * Create a latch if the client does not have async proc support.
041   * This uses the default 1.1 version.
042   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
043   */
044  public static ProcedurePrepareLatch createLatch() {
045    // don't use the latch if we have procedure support (default 1.1)
046    return createLatch(1, 1);
047  }
048
049  /**
050   * Create a latch if the client does not have async proc support
051   * @param major major version with async proc support
052   * @param minor minor version with async proc support
053   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
054   */
055  public static ProcedurePrepareLatch createLatch(int major, int minor) {
056    // don't use the latch if we have procedure support
057    return hasProcedureSupport(major, minor) ? noopLatch : new CompatibilityLatch();
058  }
059
060  /**
061   * Creates a latch which blocks.
062   */
063  public static ProcedurePrepareLatch createBlockingLatch() {
064    return new CompatibilityLatch();
065  }
066
067  /**
068   * Returns the singleton latch which does nothing.
069   */
070  public static ProcedurePrepareLatch getNoopLatch() {
071    return noopLatch;
072  }
073
074  private static boolean hasProcedureSupport(int major, int minor) {
075    return VersionInfoUtil.currentClientHasMinimumVersion(major, minor);
076  }
077
078  protected abstract void countDown(final Procedure proc);
079  public abstract void await() throws IOException;
080
081  protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
082    if (latch != null) {
083      latch.countDown(proc);
084    }
085  }
086
087  private static class NoopLatch extends ProcedurePrepareLatch {
088    @Override
089    protected void countDown(final Procedure proc) {}
090    @Override
091    public void await() throws IOException {}
092  }
093
094  protected static class CompatibilityLatch extends ProcedurePrepareLatch {
095    private final CountDownLatch latch = new CountDownLatch(1);
096
097    private IOException exception = null;
098
099    @Override
100    protected void countDown(final Procedure proc) {
101      if (proc.hasException()) {
102        exception = MasterProcedureUtil.unwrapRemoteIOException(proc);
103      }
104      latch.countDown();
105    }
106
107    @Override
108    public void await() throws IOException {
109      try {
110        latch.await();
111      } catch (InterruptedException e) {
112        throw (InterruptedIOException)new InterruptedIOException().initCause(e);
113      }
114
115      if (exception != null) {
116        throw exception;
117      }
118    }
119  }
120}