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.errorhandling;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.List;
023import org.apache.yetus.audience.InterfaceAudience;
024
025import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
026import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage;
027import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
028
029/**
030 * A ForeignException is an exception from another thread or process.
031 * <p>
032 * ForeignExceptions are sent to 'remote' peers to signal an abort in the face of failures. When
033 * serialized for transmission we encode using Protobufs to ensure version compatibility.
034 * <p>
035 * Foreign exceptions contain a Throwable as its cause. This can be a "regular" exception generated
036 * locally or a ProxyThrowable that is a representation of the original exception created on
037 * original 'remote' source. These ProxyThrowables have their their stacks traces and messages
038 * overridden to reflect the original 'remote' exception. The only way these ProxyThrowables are
039 * generated are by this class's {@link #deserialize(byte[])} method.
040 */
041@InterfaceAudience.Public
042@SuppressWarnings("serial")
043public class ForeignException extends IOException {
044
045  /**
046   * Name of the throwable's source such as a host or thread name. Must be non-null.
047   */
048  private final String source;
049
050  /**
051   * Create a new ForeignException that can be serialized. It is assumed that this came form a local
052   * source.
053   */
054  public ForeignException(String source, Throwable cause) {
055    super(cause);
056    assert source != null;
057    assert cause != null;
058    this.source = source;
059  }
060
061  /**
062   * Create a new ForeignException that can be serialized. It is assumed that this is locally
063   * generated.
064   */
065  public ForeignException(String source, String msg) {
066    super(new IllegalArgumentException(msg));
067    this.source = source;
068  }
069
070  public String getSource() {
071    return source;
072  }
073
074  /**
075   * The cause of a ForeignException can be an exception that was generated on a local in process
076   * thread, or a thread from a 'remote' separate process. If the cause is a ProxyThrowable, we know
077   * it came from deserialization which usually means it came from not only another thread, but also
078   * from a remote thread.
079   * @return true if went through deserialization, false if locally generated
080   */
081  public boolean isRemote() {
082    return getCause() instanceof ProxyThrowable;
083  }
084
085  @Override
086  public String toString() {
087    String className = getCause().getClass().getName();
088    return className + " via " + getSource() + ":" + getLocalizedMessage();
089  }
090
091  /**
092   * Convert a stack trace to list of {@link StackTraceElement}.
093   * @param trace the stack trace to convert to protobuf message
094   * @return <tt>null</tt> if the passed stack is <tt>null</tt>.
095   */
096  private static List<StackTraceElementMessage>
097    toStackTraceElementMessages(StackTraceElement[] trace) {
098    // if there is no stack trace, ignore it and just return the message
099    if (trace == null) return null;
100    // build the stack trace for the message
101    List<StackTraceElementMessage> pbTrace = new ArrayList<>(trace.length);
102    for (StackTraceElement elem : trace) {
103      StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder();
104      stackBuilder.setDeclaringClass(elem.getClassName());
105      stackBuilder.setFileName(elem.getFileName());
106      stackBuilder.setLineNumber(elem.getLineNumber());
107      stackBuilder.setMethodName(elem.getMethodName());
108      pbTrace.add(stackBuilder.build());
109    }
110    return pbTrace;
111  }
112
113  /**
114   * This is a Proxy Throwable that contains the information of the original remote exception
115   */
116  private static class ProxyThrowable extends Throwable {
117    ProxyThrowable(String msg, StackTraceElement[] trace) {
118      super(msg);
119      this.setStackTrace(trace);
120    }
121  }
122
123  /**
124   * Converts a ForeignException to an array of bytes.
125   * @param source the name of the external exception source
126   * @param t      the "local" external exception (local)
127   * @return protobuf serialized version of ForeignException
128   */
129  public static byte[] serialize(String source, Throwable t) {
130    GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder();
131    gemBuilder.setClassName(t.getClass().getName());
132    if (t.getMessage() != null) {
133      gemBuilder.setMessage(t.getMessage());
134    }
135    // set the stack trace, if there is one
136    List<StackTraceElementMessage> stack =
137      ForeignException.toStackTraceElementMessages(t.getStackTrace());
138    if (stack != null) {
139      gemBuilder.addAllTrace(stack);
140    }
141    GenericExceptionMessage payload = gemBuilder.build();
142    ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder();
143    exception.setGenericException(payload).setSource(source);
144    ForeignExceptionMessage eem = exception.build();
145    return eem.toByteArray();
146  }
147
148  /**
149   * Takes a series of bytes and tries to generate an ForeignException instance for it.
150   * @return the ForeignExcpetion instance
151   * @throws InvalidProtocolBufferException if there was deserialization problem this is thrown.
152   */
153  public static ForeignException deserialize(byte[] bytes) throws IOException {
154    // figure out the data we need to pass
155    ForeignExceptionMessage eem = ForeignExceptionMessage.parseFrom(bytes);
156    GenericExceptionMessage gem = eem.getGenericException();
157    StackTraceElement[] trace = ForeignException.toStackTrace(gem.getTraceList());
158    ProxyThrowable dfe = new ProxyThrowable(gem.getMessage(), trace);
159    ForeignException e = new ForeignException(eem.getSource(), dfe);
160    return e;
161  }
162
163  /**
164   * Unwind a serialized array of {@link StackTraceElementMessage}s to a {@link StackTraceElement}s.
165   * @param traceList list that was serialized
166   * @return the deserialized list or <tt>null</tt> if it couldn't be unwound (e.g. wasn't set on
167   *         the sender).
168   */
169  private static StackTraceElement[] toStackTrace(List<StackTraceElementMessage> traceList) {
170    if (traceList == null || traceList.isEmpty()) {
171      return new StackTraceElement[0]; // empty array
172    }
173    StackTraceElement[] trace = new StackTraceElement[traceList.size()];
174    for (int i = 0; i < traceList.size(); i++) {
175      StackTraceElementMessage elem = traceList.get(i);
176      trace[i] = new StackTraceElement(elem.getDeclaringClass(), elem.getMethodName(),
177        elem.getFileName(), elem.getLineNumber());
178    }
179    return trace;
180  }
181}