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 */
018package org.apache.hadoop.hbase.master.procedure;
019
020import java.io.IOException;
021import java.io.InterruptedIOException;
022import java.util.concurrent.CountDownLatch;
023import org.apache.hadoop.hbase.client.VersionInfoUtil;
024import org.apache.hadoop.hbase.procedure2.Procedure;
025import org.apache.yetus.audience.InterfaceAudience;
026import org.apache.yetus.audience.InterfaceStability;
027
028/**
029 * Latch used by the Master to have the prepare() sync behaviour for old clients, that can only get
030 * exceptions in a synchronous way.
031 */
032@InterfaceAudience.Private
033@InterfaceStability.Evolving
034public abstract class ProcedurePrepareLatch {
035  private static final NoopLatch noopLatch = new NoopLatch();
036
037  /**
038   * Create a latch if the client does not have async proc support. This uses the default 1.1
039   * version.
040   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
041   */
042  public static ProcedurePrepareLatch createLatch() {
043    // don't use the latch if we have procedure support (default 1.1)
044    return createLatch(1, 1);
045  }
046
047  /**
048   * Create a latch if the client does not have async proc support
049   * @param major major version with async proc support
050   * @param minor minor version with async proc support
051   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
052   */
053  public static ProcedurePrepareLatch createLatch(int major, int minor) {
054    // don't use the latch if we have procedure support
055    return hasProcedureSupport(major, minor) ? noopLatch : new CompatibilityLatch();
056  }
057
058  /**
059   * Creates a latch which blocks.
060   */
061  public static ProcedurePrepareLatch createBlockingLatch() {
062    return new CompatibilityLatch();
063  }
064
065  /**
066   * Returns the singleton latch which does nothing.
067   */
068  public static ProcedurePrepareLatch getNoopLatch() {
069    return noopLatch;
070  }
071
072  private static boolean hasProcedureSupport(int major, int minor) {
073    return VersionInfoUtil.currentClientHasMinimumVersion(major, minor);
074  }
075
076  protected abstract void countDown(final Procedure proc);
077
078  public abstract void await() throws IOException;
079
080  public static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
081    if (latch != null) {
082      latch.countDown(proc);
083    }
084  }
085
086  private static class NoopLatch extends ProcedurePrepareLatch {
087    @Override
088    protected void countDown(final Procedure proc) {
089    }
090
091    @Override
092    public void await() throws IOException {
093    }
094  }
095
096  protected static class CompatibilityLatch extends ProcedurePrepareLatch {
097    private final CountDownLatch latch = new CountDownLatch(1);
098
099    private IOException exception = null;
100
101    @Override
102    protected void countDown(final Procedure proc) {
103      if (proc.hasException()) {
104        exception = MasterProcedureUtil.unwrapRemoteIOException(proc);
105      }
106      latch.countDown();
107    }
108
109    @Override
110    public void await() throws IOException {
111      try {
112        latch.await();
113      } catch (InterruptedException e) {
114        throw (InterruptedIOException) new InterruptedIOException().initCause(e);
115      }
116
117      if (exception != null) {
118        throw exception;
119      }
120    }
121  }
122}