diff --git a/hbase-connectors-protocal-shade/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java b/hbase-connectors-protocal-shade/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java deleted file mode 100644 index 28254194..00000000 --- a/hbase-connectors-protocal-shade/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java +++ /dev/null @@ -1,149 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.util; - -import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; -import java.util.List; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage; - -/** - * Helper to convert Exceptions and StackTraces from/to protobuf. - * (see ErrorHandling.proto for the internal of the proto messages) - */ -@InterfaceAudience.Private -public final class ForeignExceptionUtil { - private ForeignExceptionUtil() { } - - public static Exception toException(final ForeignExceptionMessage eem) { - Exception re; - try { - re = createException(Exception.class, eem); - } catch (Throwable e) { - re = new Exception(eem.getGenericException().getMessage()); - } - return setExceptionDetails(re, eem); - } - - public static IOException toIOException(final ForeignExceptionMessage eem) { - IOException re; - try { - re = createException(IOException.class, eem); - } catch (Throwable e) { - re = new IOException(eem.getGenericException().getMessage()); - } - return setExceptionDetails(re, eem); - } - - private static T createException(final Class clazz, - final ForeignExceptionMessage eem) throws ClassNotFoundException, NoSuchMethodException, - InstantiationException, IllegalAccessException, InvocationTargetException { - final GenericExceptionMessage gem = eem.getGenericException(); - final Class realClass = Class.forName(gem.getClassName()); - final Class cls = realClass.asSubclass(clazz); - final Constructor cn = cls.getConstructor(String.class); - cn.setAccessible(true); - return cn.newInstance(gem.getMessage()); - } - - private static T setExceptionDetails(final T exception, - final ForeignExceptionMessage eem) { - final GenericExceptionMessage gem = eem.getGenericException(); - final StackTraceElement[] trace = toStackTrace(gem.getTraceList()); - exception.setStackTrace(trace); - return exception; - } - - public static ForeignExceptionMessage toProtoForeignException(final Throwable t) { - return toProtoForeignException(null, t); - } - - public static ForeignExceptionMessage toProtoForeignException(String source, Throwable t) { - GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder(); - gemBuilder.setClassName(t.getClass().getName()); - if (t.getMessage() != null) { - gemBuilder.setMessage(t.getMessage()); - } - // set the stack trace, if there is one - List stack = toProtoStackTraceElement(t.getStackTrace()); - if (stack != null) { - gemBuilder.addAllTrace(stack); - } - GenericExceptionMessage payload = gemBuilder.build(); - ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder(); - exception.setGenericException(payload); - if (source != null) { - exception.setSource(source); - } - - return exception.build(); - } - - /** - * Convert a stack trace to list of {@link StackTraceElement}. - * @param trace the stack trace to convert to protobuf message - * @return null if the passed stack is null. - */ - public static List toProtoStackTraceElement(StackTraceElement[] trace) { - // if there is no stack trace, ignore it and just return the message - if (trace == null) { - return null; - } - - // build the stack trace for the message - List pbTrace = new ArrayList<>(trace.length); - for (StackTraceElement elem : trace) { - StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder(); - stackBuilder.setDeclaringClass(elem.getClassName()); - if (elem.getFileName() != null) { - stackBuilder.setFileName(elem.getFileName()); - } - stackBuilder.setLineNumber(elem.getLineNumber()); - stackBuilder.setMethodName(elem.getMethodName()); - pbTrace.add(stackBuilder.build()); - } - return pbTrace; - } - - /** - * Unwind a serialized array of {@link StackTraceElementMessage}s to a - * {@link StackTraceElement}s. - * @param traceList list that was serialized - * @return the deserialized list or null if it couldn't be unwound (e.g. wasn't set on - * the sender). - */ - public static StackTraceElement[] toStackTrace(List traceList) { - if (traceList == null || traceList.isEmpty()) { - return new StackTraceElement[0]; // empty array - } - StackTraceElement[] trace = new StackTraceElement[traceList.size()]; - for (int i = 0; i < traceList.size(); i++) { - StackTraceElementMessage elem = traceList.get(i); - trace[i] = new StackTraceElement( - elem.getDeclaringClass(), elem.getMethodName(), - elem.hasFileName() ? elem.getFileName() : null, - elem.getLineNumber()); - } - return trace; - } -} \ No newline at end of file diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/ErrorHandling.proto b/hbase-connectors-protocal-shade/src/main/protobuf/ErrorHandling.proto deleted file mode 100644 index 367fd85f..00000000 --- a/hbase-connectors-protocal-shade/src/main/protobuf/ErrorHandling.proto +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -syntax = "proto2"; - -// This file contains protocol buffers that are used for error handling -package hbase.pb; - -option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated"; -option java_outer_classname = "ErrorHandlingProtos"; -option java_generate_equals_and_hash = true; -option optimize_for = SPEED; - -/** - * Protobuf version of a java.lang.StackTraceElement - * so we can serialize exceptions. - */ -message StackTraceElementMessage { - optional string declaring_class = 1; - optional string method_name = 2; - optional string file_name = 3; - optional int32 line_number = 4; -} - -/** - * Cause of a remote failure for a generic exception. Contains - * all the information for a generic exception as well as - * optional info about the error for generic info passing - * (which should be another protobuffed class). - */ -message GenericExceptionMessage { - optional string class_name = 1; - optional string message = 2; - optional bytes error_info = 3; - repeated StackTraceElementMessage trace = 4; -} - -/** - * Exception sent across the wire when a remote task needs - * to notify other tasks that it failed and why - */ -message ForeignExceptionMessage { - optional string source = 1; - optional GenericExceptionMessage generic_exception = 2; -} diff --git a/hbase-connectors-protocal-shade/pom.xml b/hbase-connectors-protocol-shaded/pom.xml similarity index 97% rename from hbase-connectors-protocal-shade/pom.xml rename to hbase-connectors-protocol-shaded/pom.xml index 7e4e93f2..32c4d832 100644 --- a/hbase-connectors-protocal-shade/pom.xml +++ b/hbase-connectors-protocol-shaded/pom.xml @@ -15,10 +15,10 @@ Shaded protobuf protocol classes used by HBase Connectors internally. true - - 3.7.1 + 3.5.1-1 @@ -189,10 +189,6 @@ org.apache.htrace htrace-core4 - - org.apache.yetus - audience-annotations - diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/CellMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/CellMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/CellMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/CellMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/CellSetMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/CellSetMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/CellSetMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/CellSetMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/ColumnSchemaMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/ColumnSchemaMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/ColumnSchemaMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/ColumnSchemaMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/NamespacePropertiesMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/NamespacePropertiesMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/NamespacePropertiesMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/NamespacePropertiesMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/NamespacesMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/NamespacesMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/NamespacesMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/NamespacesMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/ScannerMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/ScannerMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/ScannerMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/ScannerMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/StorageClusterStatusMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/StorageClusterStatusMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/StorageClusterStatusMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/StorageClusterStatusMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/TableInfoMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableInfoMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/TableInfoMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableInfoMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/TableListMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableListMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/TableListMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableListMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/TableSchemaMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableSchemaMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/TableSchemaMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/TableSchemaMessage.proto diff --git a/hbase-connectors-protocal-shade/src/main/protobuf/rest/VersionMessage.proto b/hbase-connectors-protocol-shaded/src/main/protobuf/rest/VersionMessage.proto similarity index 100% rename from hbase-connectors-protocal-shade/src/main/protobuf/rest/VersionMessage.proto rename to hbase-connectors-protocol-shaded/src/main/protobuf/rest/VersionMessage.proto diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml index 3c3ef7e0..e700546b 100644 --- a/hbase-rest/pom.xml +++ b/hbase-rest/pom.xml @@ -132,17 +132,14 @@ org.apache.hbase - hbase-protocol-shaded + hbase-connectors-protocol-shaded + ${revision} jar org.apache.hbase hbase-client - - org.apache.hbase - hbase-server - org.apache.hbase hbase-hadoop-compat diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java index faa9d666..ae91d11e 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java @@ -30,14 +30,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.http.ClickjackingPreventionFilter; -import org.apache.hadoop.hbase.http.HttpServerUtil; -import org.apache.hadoop.hbase.http.InfoServer; -import org.apache.hadoop.hbase.http.SecurityHeadersFilter; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.rest.filter.AuthFilter; import org.apache.hadoop.hbase.rest.filter.GzipFilter; import org.apache.hadoop.hbase.rest.filter.RestCsrfPreventionFilter; +import org.apache.hadoop.hbase.rest.http.ClickjackingPreventionFilter; +import org.apache.hadoop.hbase.rest.http.HttpServerUtil; +import org.apache.hadoop.hbase.rest.http.InfoServer; +import org.apache.hadoop.hbase.rest.http.SecurityHeadersFilter; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.DNS; import org.apache.hadoop.hbase.util.Pair; diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java index 6c71bb62..4213fa9c 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java @@ -21,6 +21,8 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.rest.util.ConnectionCache; +import org.apache.hadoop.hbase.rest.util.JvmPauseMonitor; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,8 +30,6 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.ParseFilter; import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.util.ConnectionCache; -import org.apache.hadoop.hbase.util.JvmPauseMonitor; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.ProxyUsers; diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/AdminAuthorizedFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/AdminAuthorizedFilter.java new file mode 100644 index 00000000..9f248241 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/AdminAuthorizedFilter.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.yetus.audience.InterfaceAudience; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; + +@InterfaceAudience.Private +public class AdminAuthorizedFilter implements Filter { + + private Configuration conf; + private AccessControlList adminsAcl; + + @Override public void init(FilterConfig filterConfig) throws ServletException { + adminsAcl = (AccessControlList) filterConfig.getServletContext().getAttribute( + HttpServer.ADMINS_ACL); + conf = (Configuration) filterConfig.getServletContext().getAttribute( + HttpServer.CONF_CONTEXT_ATTRIBUTE); + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws IOException, ServletException { + if (!(request instanceof HttpServletRequest) || !(response instanceof HttpServletResponse)) { + throw new UnsupportedOperationException("Only accepts HTTP"); + } + HttpServletRequest httpReq = (HttpServletRequest) request; + HttpServletResponse httpResp = (HttpServletResponse) response; + + if (!HttpServer.hasAdministratorAccess(conf, adminsAcl, httpReq, httpResp)) { + return; + } + + chain.doFilter(request, response); + } + + @Override public void destroy() {} +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ClickjackingPreventionFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ClickjackingPreventionFilter.java new file mode 100644 index 00000000..0db32b2e --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ClickjackingPreventionFilter.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.yetus.audience.InterfaceAudience; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class ClickjackingPreventionFilter implements Filter { + private FilterConfig filterConfig; + private static final String DEFAULT_XFRAMEOPTIONS = "DENY"; + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + this.filterConfig = filterConfig; + } + + @Override + public void doFilter(ServletRequest req, ServletResponse res, FilterChain chain) + throws IOException, ServletException { + HttpServletResponse httpRes = (HttpServletResponse) res; + httpRes.addHeader("X-Frame-Options", filterConfig.getInitParameter("xframeoptions")); + chain.doFilter(req, res); + } + + @Override + public void destroy() { + } + + public static Map getDefaultParameters(Configuration conf) { + Map params = new HashMap<>(); + params.put("xframeoptions", conf.get("hbase.http.filter.xframeoptions.mode", + DEFAULT_XFRAMEOPTIONS)); + return params; + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HtmlQuoting.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HtmlQuoting.java new file mode 100644 index 00000000..b47357a3 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HtmlQuoting.java @@ -0,0 +1,230 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +/** + * This class is responsible for quoting HTML characters. + */ +@InterfaceAudience.Private +public final class HtmlQuoting { + private static final byte[] ampBytes = Bytes.toBytes("&"); + private static final byte[] aposBytes = Bytes.toBytes("'"); + private static final byte[] gtBytes = Bytes.toBytes(">"); + private static final byte[] ltBytes = Bytes.toBytes("<"); + private static final byte[] quotBytes = Bytes.toBytes("""); + + /** + * Does the given string need to be quoted? + * @param data the string to check + * @param off the starting position + * @param len the number of bytes to check + * @return does the string contain any of the active html characters? + */ + public static boolean needsQuoting(byte[] data, int off, int len) { + if (off+len > data.length) { + throw new IllegalStateException("off+len=" + off+len + " should be lower" + + " than data length=" + data.length); + } + for(int i=off; i< off+len; ++i) { + switch(data[i]) { + case '&': + case '<': + case '>': + case '\'': + case '"': + return true; + default: + break; + } + } + return false; + } + + /** + * Does the given string need to be quoted? + * @param str the string to check + * @return does the string contain any of the active html characters? + */ + public static boolean needsQuoting(String str) { + if (str == null) { + return false; + } + byte[] bytes = Bytes.toBytes(str); + return needsQuoting(bytes, 0 , bytes.length); + } + + /** + * Quote all of the active HTML characters in the given string as they + * are added to the buffer. + * @param output the stream to write the output to + * @param buffer the byte array to take the characters from + * @param off the index of the first byte to quote + * @param len the number of bytes to quote + */ + public static void quoteHtmlChars(OutputStream output, byte[] buffer, int off, int len) + throws IOException { + for(int i=off; i < off+len; i++) { + switch (buffer[i]) { + case '&': + output.write(ampBytes); + break; + case '<': + output.write(ltBytes); + break; + case '>': + output.write(gtBytes); + break; + case '\'': + output.write(aposBytes); + break; + case '"': + output.write(quotBytes); + break; + default: + output.write(buffer, i, 1); + break; + } + } + } + + /** + * Quote the given item to make it html-safe. + * @param item the string to quote + * @return the quoted string + */ + public static String quoteHtmlChars(String item) { + if (item == null) { + return null; + } + byte[] bytes = Bytes.toBytes(item); + if (needsQuoting(bytes, 0, bytes.length)) { + ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + try { + quoteHtmlChars(buffer, bytes, 0, bytes.length); + } catch (IOException ioe) { + // Won't happen, since it is a bytearrayoutputstream + } + return buffer.toString(); + } else { + return item; + } + } + + /** + * Return an output stream that quotes all of the output. + * @param out the stream to write the quoted output to + * @return a new stream that the application show write to + */ + public static OutputStream quoteOutputStream(final OutputStream out) { + return new OutputStream() { + private byte[] data = new byte[1]; + @Override + public void write(byte[] data, int off, int len) throws IOException { + quoteHtmlChars(out, data, off, len); + } + + @Override + public void write(int b) throws IOException { + data[0] = (byte) b; + quoteHtmlChars(out, data, 0, 1); + } + + @Override + public void flush() throws IOException { + out.flush(); + } + + @Override + public void close() throws IOException { + out.close(); + } + }; + } + + /** + * Remove HTML quoting from a string. + * @param item the string to unquote + * @return the unquoted string + */ + public static String unquoteHtmlChars(String item) { + if (item == null) { + return null; + } + int next = item.indexOf('&'); + // nothing was quoted + if (next == -1) { + return item; + } + int len = item.length(); + int posn = 0; + StringBuilder buffer = new StringBuilder(); + while (next != -1) { + buffer.append(item.substring(posn, next)); + if (item.startsWith("&", next)) { + buffer.append('&'); + next += 5; + } else if (item.startsWith("'", next)) { + buffer.append('\''); + next += 6; + } else if (item.startsWith(">", next)) { + buffer.append('>'); + next += 4; + } else if (item.startsWith("<", next)) { + buffer.append('<'); + next += 4; + } else if (item.startsWith(""", next)) { + buffer.append('"'); + next += 6; + } else { + int end = item.indexOf(';', next)+1; + if (end == 0) { + end = len; + } + throw new IllegalArgumentException("Bad HTML quoting for " + + item.substring(next,end)); + } + posn = next; + next = item.indexOf('&', posn); + } + buffer.append(item.substring(posn, len)); + return buffer.toString(); + } + + public static void main(String[] args) { + if (args.length == 0) { + throw new IllegalArgumentException("Please provide some arguments"); + } + for(String arg:args) { + System.out.println("Original: " + arg); + String quoted = quoteHtmlChars(arg); + System.out.println("Quoted: "+ quoted); + String unquoted = unquoteHtmlChars(quoted); + System.out.println("Unquoted: " + unquoted); + System.out.println(); + } + } + + private HtmlQuoting() {} +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpConfig.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpConfig.java new file mode 100644 index 00000000..e27b0900 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpConfig.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +/** + * Statics to get access to Http related configuration. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class HttpConfig { + private Policy policy; + public enum Policy { + HTTP_ONLY, + HTTPS_ONLY, + HTTP_AND_HTTPS; + + public Policy fromString(String value) { + if (HTTPS_ONLY.name().equalsIgnoreCase(value)) { + return HTTPS_ONLY; + } else if (HTTP_AND_HTTPS.name().equalsIgnoreCase(value)) { + return HTTP_AND_HTTPS; + } + return HTTP_ONLY; + } + + public boolean isHttpEnabled() { + return this == HTTP_ONLY || this == HTTP_AND_HTTPS; + } + + public boolean isHttpsEnabled() { + return this == HTTPS_ONLY || this == HTTP_AND_HTTPS; + } + } + + public HttpConfig(final Configuration conf) { + boolean sslEnabled = conf.getBoolean( + ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY, + ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT); + policy = sslEnabled ? Policy.HTTPS_ONLY : Policy.HTTP_ONLY; + if (sslEnabled) { + conf.addResource("ssl-server.xml"); + conf.addResource("ssl-client.xml"); + } + } + + public void setPolicy(Policy policy) { + this.policy = policy; + } + + public boolean isSecure() { + return policy == Policy.HTTPS_ONLY; + } + + public String getSchemePrefix() { + return (isSecure()) ? "https://" : "http://"; + } + + public String getScheme(Policy policy) { + return policy == Policy.HTTPS_ONLY ? "https://" : "http://"; + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServer.java new file mode 100644 index 00000000..61c3e76d --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServer.java @@ -0,0 +1,1508 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.rest.http.conf.ConfServlet; +import org.apache.hadoop.hbase.rest.http.jmx.JMXJsonServlet; +import org.apache.hadoop.hbase.rest.http.log.LogLevel; +import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.http.AdminAuthorizedServlet; +import org.apache.hadoop.http.FilterContainer; +import org.apache.hadoop.http.FilterInitializer; +import org.apache.hadoop.http.HttpRequestLog; +import org.apache.hadoop.http.NoCacheFilter; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.server.AuthenticationFilter; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hadoop.util.Shell; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.eclipse.jetty.http.HttpVersion; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.SecureRequestCustomizer; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; +import org.eclipse.jetty.server.handler.ContextHandlerCollection; +import org.eclipse.jetty.server.handler.HandlerCollection; +import org.eclipse.jetty.server.handler.RequestLogHandler; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.FilterMapping; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.MultiException; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.eclipse.jetty.util.thread.QueuedThreadPool; +import org.eclipse.jetty.webapp.WebAppContext; +import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.servlet.ServletContainer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletRequestWrapper; +import javax.servlet.http.HttpServletResponse; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.io.PrintStream; +import java.net.BindException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Create a Jetty embedded server to answer http requests. The primary goal + * is to serve up status information for the server. + * There are three contexts: + * "/logs/" -> points to the log directory + * "/static/" -> points to common static files (src/webapps/static) + * "/" -> the jsp server code from (src/webapps/<name>) + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class HttpServer implements FilterContainer { + private static final Logger LOG = LoggerFactory.getLogger(HttpServer.class); + private static final String EMPTY_STRING = ""; + + private static final int DEFAULT_MAX_HEADER_SIZE = 64 * 1024; // 64K + + static final String FILTER_INITIALIZERS_PROPERTY + = "hbase.http.filter.initializers"; + static final String HTTP_MAX_THREADS = "hbase.http.max.threads"; + + public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui"; + static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication."; + static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX + + "spnego."; + static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal"; + public static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX; + static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX = "kerberos.keytab"; + public static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX; + static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX = "kerberos.name.rules"; + public static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX; + static final String HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_SUFFIX = "kerberos.proxyuser.enable"; + public static final String HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_SUFFIX; + public static final boolean HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_DEFAULT = false; + static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX = + "signature.secret.file"; + public static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY = + HTTP_AUTHENTICATION_PREFIX + HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX; + public static final String HTTP_SPNEGO_AUTHENTICATION_ADMIN_USERS_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + "admin.users"; + public static final String HTTP_SPNEGO_AUTHENTICATION_ADMIN_GROUPS_KEY = + HTTP_SPNEGO_AUTHENTICATION_PREFIX + "admin.groups"; + public static final String HTTP_PRIVILEGED_CONF_KEY = + "hbase.security.authentication.ui.config.protected"; + public static final boolean HTTP_PRIVILEGED_CONF_DEFAULT = false; + + // The ServletContext attribute where the daemon Configuration + // gets stored. + public static final String CONF_CONTEXT_ATTRIBUTE = "hbase.conf"; + public static final String ADMINS_ACL = "admins.acl"; + public static final String BIND_ADDRESS = "bind.address"; + public static final String SPNEGO_FILTER = "SpnegoFilter"; + public static final String SPNEGO_PROXYUSER_FILTER = "SpnegoProxyUserFilter"; + public static final String NO_CACHE_FILTER = "NoCacheFilter"; + public static final String APP_DIR = "webapps"; + + private final AccessControlList adminsAcl; + + protected final Server webServer; + protected String appDir; + protected String logDir; + + private static final class ListenerInfo { + /** + * Boolean flag to determine whether the HTTP server should clean up the + * listener in stop(). + */ + private final boolean isManaged; + private final ServerConnector listener; + private ListenerInfo(boolean isManaged, ServerConnector listener) { + this.isManaged = isManaged; + this.listener = listener; + } + } + + private final List listeners = Lists.newArrayList(); + + @VisibleForTesting + public List getServerConnectors() { + return listeners.stream().map(info -> info.listener).collect(Collectors.toList()); + } + + protected final WebAppContext webAppContext; + protected final boolean findPort; + protected final Map defaultContexts = new HashMap<>(); + protected final List filterNames = new ArrayList<>(); + protected final boolean authenticationEnabled; + static final String STATE_DESCRIPTION_ALIVE = " - alive"; + static final String STATE_DESCRIPTION_NOT_LIVE = " - not live"; + + /** + * Class to construct instances of HTTP server with specific options. + */ + public static class Builder { + private ArrayList endpoints = Lists.newArrayList(); + private Configuration conf; + private String[] pathSpecs; + private AccessControlList adminsAcl; + private boolean securityEnabled = false; + private String usernameConfKey; + private String keytabConfKey; + private boolean needsClientAuth; + + private String hostName; + private String appDir = APP_DIR; + private String logDir; + private boolean findPort; + + private String trustStore; + private String trustStorePassword; + private String trustStoreType; + + private String keyStore; + private String keyStorePassword; + private String keyStoreType; + + // The -keypass option in keytool + private String keyPassword; + + private String kerberosNameRulesKey; + private String signatureSecretFileKey; + + /** + * @see #setAppDir(String) + * @deprecated Since 0.99.0. Use builder pattern via {@link #setAppDir(String)} instead. + */ + @Deprecated + private String name; + /** + * @see #addEndpoint(URI) + * @deprecated Since 0.99.0. Use builder pattern via {@link #addEndpoint(URI)} instead. + */ + @Deprecated + private String bindAddress; + /** + * @see #addEndpoint(URI) + * @deprecated Since 0.99.0. Use builder pattern via {@link #addEndpoint(URI)} instead. + */ + @Deprecated + private int port = -1; + + /** + * Add an endpoint that the HTTP server should listen to. + * + * @param endpoint + * the endpoint of that the HTTP server should listen to. The + * scheme specifies the protocol (i.e. HTTP / HTTPS), the host + * specifies the binding address, and the port specifies the + * listening port. Unspecified or zero port means that the server + * can listen to any port. + */ + public Builder addEndpoint(URI endpoint) { + endpoints.add(endpoint); + return this; + } + + /** + * Set the hostname of the http server. The host name is used to resolve the + * _HOST field in Kerberos principals. The hostname of the first listener + * will be used if the name is unspecified. + */ + public Builder hostName(String hostName) { + this.hostName = hostName; + return this; + } + + public Builder trustStore(String location, String password, String type) { + this.trustStore = location; + this.trustStorePassword = password; + this.trustStoreType = type; + return this; + } + + public Builder keyStore(String location, String password, String type) { + this.keyStore = location; + this.keyStorePassword = password; + this.keyStoreType = type; + return this; + } + + public Builder keyPassword(String password) { + this.keyPassword = password; + return this; + } + + /** + * Specify whether the server should authorize the client in SSL + * connections. + */ + public Builder needsClientAuth(boolean value) { + this.needsClientAuth = value; + return this; + } + + /** + * @see #setAppDir(String) + * @deprecated Since 0.99.0. Use {@link #setAppDir(String)} instead. + */ + @Deprecated + public Builder setName(String name){ + this.name = name; + return this; + } + + /** + * @see #addEndpoint(URI) + * @deprecated Since 0.99.0. Use {@link #addEndpoint(URI)} instead. + */ + @Deprecated + public Builder setBindAddress(String bindAddress){ + this.bindAddress = bindAddress; + return this; + } + + /** + * @see #addEndpoint(URI) + * @deprecated Since 0.99.0. Use {@link #addEndpoint(URI)} instead. + */ + @Deprecated + public Builder setPort(int port) { + this.port = port; + return this; + } + + public Builder setFindPort(boolean findPort) { + this.findPort = findPort; + return this; + } + + public Builder setConf(Configuration conf) { + this.conf = conf; + return this; + } + + public Builder setPathSpec(String[] pathSpec) { + this.pathSpecs = pathSpec; + return this; + } + + public Builder setACL(AccessControlList acl) { + this.adminsAcl = acl; + return this; + } + + public Builder setSecurityEnabled(boolean securityEnabled) { + this.securityEnabled = securityEnabled; + return this; + } + + public Builder setUsernameConfKey(String usernameConfKey) { + this.usernameConfKey = usernameConfKey; + return this; + } + + public Builder setKeytabConfKey(String keytabConfKey) { + this.keytabConfKey = keytabConfKey; + return this; + } + + public Builder setKerberosNameRulesKey(String kerberosNameRulesKey) { + this.kerberosNameRulesKey = kerberosNameRulesKey; + return this; + } + + public Builder setSignatureSecretFileKey(String signatureSecretFileKey) { + this.signatureSecretFileKey = signatureSecretFileKey; + return this; + } + + public Builder setAppDir(String appDir) { + this.appDir = appDir; + return this; + } + + public Builder setLogDir(String logDir) { + this.logDir = logDir; + return this; + } + + public HttpServer build() throws IOException { + + // Do we still need to assert this non null name if it is deprecated? + if (this.name == null) { + throw new HadoopIllegalArgumentException("name is not set"); + } + + // Make the behavior compatible with deprecated interfaces + if (bindAddress != null && port != -1) { + try { + endpoints.add(0, new URI("http", "", bindAddress, port, "", "", "")); + } catch (URISyntaxException e) { + throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e); + } + } + + if (endpoints.isEmpty()) { + throw new HadoopIllegalArgumentException("No endpoints specified"); + } + + if (hostName == null) { + hostName = endpoints.get(0).getHost(); + } + + if (this.conf == null) { + conf = new Configuration(); + } + + HttpServer server = new HttpServer(this); + + for (URI ep : endpoints) { + ServerConnector listener = null; + String scheme = ep.getScheme(); + HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setSecureScheme("https"); + httpConfig.setHeaderCacheSize(DEFAULT_MAX_HEADER_SIZE); + httpConfig.setResponseHeaderSize(DEFAULT_MAX_HEADER_SIZE); + httpConfig.setRequestHeaderSize(DEFAULT_MAX_HEADER_SIZE); + + if ("http".equals(scheme)) { + listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig)); + } else if ("https".equals(scheme)) { + HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig); + httpsConfig.addCustomizer(new SecureRequestCustomizer()); + SslContextFactory sslCtxFactory = new SslContextFactory(); + sslCtxFactory.setNeedClientAuth(needsClientAuth); + sslCtxFactory.setKeyManagerPassword(keyPassword); + + if (keyStore != null) { + sslCtxFactory.setKeyStorePath(keyStore); + sslCtxFactory.setKeyStoreType(keyStoreType); + sslCtxFactory.setKeyStorePassword(keyStorePassword); + } + + if (trustStore != null) { + sslCtxFactory.setTrustStorePath(trustStore); + sslCtxFactory.setTrustStoreType(trustStoreType); + sslCtxFactory.setTrustStorePassword(trustStorePassword); + + } + listener = new ServerConnector(server.webServer, new SslConnectionFactory(sslCtxFactory, + HttpVersion.HTTP_1_1.toString()), new HttpConnectionFactory(httpsConfig)); + } else { + throw new HadoopIllegalArgumentException( + "unknown scheme for endpoint:" + ep); + } + + // default settings for connector + listener.setAcceptQueueSize(128); + if (Shell.WINDOWS) { + // result of setting the SO_REUSEADDR flag is different on Windows + // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx + // without this 2 NN's can start on the same machine and listen on + // the same port with indeterminate routing of incoming requests to them + listener.setReuseAddress(false); + } + + listener.setHost(ep.getHost()); + listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort()); + server.addManagedListener(listener); + } + + server.loadListeners(); + return server; + + } + + } + + /** + * @see #HttpServer(String, String, int, boolean, Configuration) + * @deprecated Since 0.99.0 + */ + @Deprecated + public HttpServer(String name, String bindAddress, int port, boolean findPort) + throws IOException { + this(name, bindAddress, port, findPort, new Configuration()); + } + + /** + * Create a status server on the given port. Allows you to specify the + * path specifications that this server will be serving so that they will be + * added to the filters properly. + * + * @param name The name of the server + * @param bindAddress The address for this server + * @param port The port to use on the server + * @param findPort whether the server should start at the given port and + * increment by 1 until it finds a free port. + * @param conf Configuration + * @param pathSpecs Path specifications that this httpserver will be serving. + * These will be added to any filters. + * @deprecated Since 0.99.0 + */ + @Deprecated + public HttpServer(String name, String bindAddress, int port, + boolean findPort, Configuration conf, String[] pathSpecs) throws IOException { + this(name, bindAddress, port, findPort, conf, null, pathSpecs); + } + + /** + * Create a status server on the given port. + * The jsp scripts are taken from src/webapps/<name>. + * @param name The name of the server + * @param port The port to use on the server + * @param findPort whether the server should start at the given port and + * increment by 1 until it finds a free port. + * @param conf Configuration + * @deprecated Since 0.99.0 + */ + @Deprecated + public HttpServer(String name, String bindAddress, int port, + boolean findPort, Configuration conf) throws IOException { + this(name, bindAddress, port, findPort, conf, null, null); + } + + /** + * Creates a status server on the given port. The JSP scripts are taken + * from src/webapp<name>. + * + * @param name the name of the server + * @param bindAddress the address for this server + * @param port the port to use on the server + * @param findPort whether the server should start at the given port and increment by 1 until it + * finds a free port + * @param conf the configuration to use + * @param adminsAcl {@link AccessControlList} of the admins + * @throws IOException when creating the server fails + * @deprecated Since 0.99.0 + */ + @Deprecated + public HttpServer(String name, String bindAddress, int port, + boolean findPort, Configuration conf, AccessControlList adminsAcl) + throws IOException { + this(name, bindAddress, port, findPort, conf, adminsAcl, null); + } + + /** + * Create a status server on the given port. + * The jsp scripts are taken from src/webapps/<name>. + * @param name The name of the server + * @param bindAddress The address for this server + * @param port The port to use on the server + * @param findPort whether the server should start at the given port and + * increment by 1 until it finds a free port. + * @param conf Configuration + * @param adminsAcl {@link AccessControlList} of the admins + * @param pathSpecs Path specifications that this httpserver will be serving. + * These will be added to any filters. + * @deprecated Since 0.99.0 + */ + @Deprecated + public HttpServer(String name, String bindAddress, int port, + boolean findPort, Configuration conf, AccessControlList adminsAcl, + String[] pathSpecs) throws IOException { + this(new Builder().setName(name) + .addEndpoint(URI.create("http://" + bindAddress + ":" + port)) + .setFindPort(findPort).setConf(conf).setACL(adminsAcl) + .setPathSpec(pathSpecs)); + } + + private HttpServer(final Builder b) throws IOException { + this.appDir = b.appDir; + this.logDir = b.logDir; + final String appDir = getWebAppsPath(b.name); + + + int maxThreads = b.conf.getInt(HTTP_MAX_THREADS, 16); + // If HTTP_MAX_THREADS is less than or equal to 0, QueueThreadPool() will use the + // default value (currently 200). + QueuedThreadPool threadPool = maxThreads <= 0 ? new QueuedThreadPool() + : new QueuedThreadPool(maxThreads); + threadPool.setDaemon(true); + this.webServer = new Server(threadPool); + + this.adminsAcl = b.adminsAcl; + this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir); + this.findPort = b.findPort; + this.authenticationEnabled = b.securityEnabled; + initializeWebServer(b.name, b.hostName, b.conf, b.pathSpecs, b); + } + + private void initializeWebServer(String name, String hostName, + Configuration conf, String[] pathSpecs, HttpServer.Builder b) + throws FileNotFoundException, IOException { + + Preconditions.checkNotNull(webAppContext); + + HandlerCollection handlerCollection = new HandlerCollection(); + + ContextHandlerCollection contexts = new ContextHandlerCollection(); + RequestLog requestLog = HttpRequestLog.getRequestLog(name); + + if (requestLog != null) { + RequestLogHandler requestLogHandler = new RequestLogHandler(); + requestLogHandler.setRequestLog(requestLog); + handlerCollection.addHandler(requestLogHandler); + } + + final String appDir = getWebAppsPath(name); + + handlerCollection.addHandler(contexts); + handlerCollection.addHandler(webAppContext); + + webServer.setHandler(handlerCollection); + + webAppContext.setAttribute(ADMINS_ACL, adminsAcl); + + // Default apps need to be set first, so that all filters are applied to them. + // Because they're added to defaultContexts, we need them there before we start + // adding filters + addDefaultApps(contexts, appDir, conf); + + addGlobalFilter("safety", QuotingInputFilter.class.getName(), null); + + addGlobalFilter("clickjackingprevention", + ClickjackingPreventionFilter.class.getName(), + ClickjackingPreventionFilter.getDefaultParameters(conf)); + + addGlobalFilter("securityheaders", + SecurityHeadersFilter.class.getName(), + SecurityHeadersFilter.getDefaultParameters(conf)); + + // But security needs to be enabled prior to adding the other servlets + if (authenticationEnabled) { + initSpnego(conf, hostName, b.usernameConfKey, b.keytabConfKey, b.kerberosNameRulesKey, + b.signatureSecretFileKey); + } + + final FilterInitializer[] initializers = getFilterInitializers(conf); + if (initializers != null) { + conf = new Configuration(conf); + conf.set(BIND_ADDRESS, hostName); + for (FilterInitializer c : initializers) { + c.initFilter(this, conf); + } + } + + addDefaultServlets(contexts, conf); + + if (pathSpecs != null) { + for (String path : pathSpecs) { + LOG.info("adding path spec: " + path); + addFilterPathMapping(path, webAppContext); + } + } + } + + private void addManagedListener(ServerConnector connector) { + listeners.add(new ListenerInfo(true, connector)); + } + + private static WebAppContext createWebAppContext(String name, + Configuration conf, AccessControlList adminsAcl, final String appDir) { + WebAppContext ctx = new WebAppContext(); + ctx.setDisplayName(name); + ctx.setContextPath("/"); + ctx.setWar(appDir + "/" + name); + ctx.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf); + // for org.apache.hadoop.metrics.MetricsServlet + ctx.getServletContext().setAttribute( + org.apache.hadoop.http.HttpServer2.CONF_CONTEXT_ATTRIBUTE, conf); + ctx.getServletContext().setAttribute(ADMINS_ACL, adminsAcl); + addNoCacheFilter(ctx); + return ctx; + } + + private static void addNoCacheFilter(WebAppContext ctxt) { + defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(), + Collections. emptyMap(), new String[] { "/*" }); + } + + /** Get an array of FilterConfiguration specified in the conf */ + private static FilterInitializer[] getFilterInitializers(Configuration conf) { + if (conf == null) { + return null; + } + + Class[] classes = conf.getClasses(FILTER_INITIALIZERS_PROPERTY); + if (classes == null) { + return null; + } + + FilterInitializer[] initializers = new FilterInitializer[classes.length]; + for(int i = 0; i < classes.length; i++) { + initializers[i] = (FilterInitializer) ReflectionUtils.newInstance(classes[i]); + } + return initializers; + } + + /** + * Add default apps. + * @param appDir The application directory + */ + protected void addDefaultApps(ContextHandlerCollection parent, + final String appDir, Configuration conf) { + // set up the context for "/logs/" if "hadoop.log.dir" property is defined. + String logDir = this.logDir; + if (logDir == null) { + logDir = System.getProperty("hadoop.log.dir"); + } + if (logDir != null) { + ServletContextHandler logContext = new ServletContextHandler(parent, "/logs"); + logContext.addServlet(AdminAuthorizedServlet.class, "/*"); + logContext.setResourceBase(logDir); + + if (conf.getBoolean( + ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES, + ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) { + Map params = logContext.getInitParams(); + params.put( + "org.mortbay.jetty.servlet.Default.aliases", "true"); + } + logContext.setDisplayName("logs"); + setContextAttributes(logContext, conf); + defaultContexts.put(logContext, true); + } + // set up the context for "/static/*" + ServletContextHandler staticContext = new ServletContextHandler(parent, "/static"); + staticContext.setResourceBase(appDir + "/static"); + staticContext.addServlet(DefaultServlet.class, "/*"); + staticContext.setDisplayName("static"); + setContextAttributes(staticContext, conf); + defaultContexts.put(staticContext, true); + } + + private void setContextAttributes(ServletContextHandler context, Configuration conf) { + context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf); + context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl); + } + + /** + * Add default servlets. + */ + protected void addDefaultServlets( + ContextHandlerCollection contexts, Configuration conf) throws IOException { + // set up default servlets + addPrivilegedServlet("stacks", "/stacks", StackServlet.class); + addPrivilegedServlet("logLevel", "/logLevel", LogLevel.Servlet.class); + // Hadoop3 has moved completely to metrics2, and dropped support for Metrics v1's + // MetricsServlet (see HADOOP-12504). We'll using reflection to load if against hadoop2. + // Remove when we drop support for hbase on hadoop2.x. + try { + Class clz = Class.forName("org.apache.hadoop.metrics.MetricsServlet"); + addPrivilegedServlet("metrics", "/metrics", clz.asSubclass(HttpServlet.class)); + } catch (Exception e) { + // do nothing + } + addPrivilegedServlet("jmx", "/jmx", JMXJsonServlet.class); + // While we don't expect users to have sensitive information in their configuration, they + // might. Give them an option to not expose the service configuration to all users. + if (conf.getBoolean(HTTP_PRIVILEGED_CONF_KEY, HTTP_PRIVILEGED_CONF_DEFAULT)) { + addPrivilegedServlet("conf", "/conf", ConfServlet.class); + } else { + addUnprivilegedServlet("conf", "/conf", ConfServlet.class); + } + final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome(); + if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) { + addPrivilegedServlet("prof", "/prof", ProfileServlet.class); + Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR); + if (Files.notExists(tmpDir)) { + Files.createDirectories(tmpDir); + } + ServletContextHandler genCtx = new ServletContextHandler(contexts, "/prof-output"); + genCtx.addServlet(ProfileOutputServlet.class, "/*"); + genCtx.setResourceBase(tmpDir.toAbsolutePath().toString()); + genCtx.setDisplayName("prof-output"); + } else { + addUnprivilegedServlet("prof", "/prof", ProfileServlet.DisabledServlet.class); + LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " + + "not specified. Disabling /prof endpoint."); + } + } + + /** + * Set a value in the webapp context. These values are available to the jsp + * pages as "application.getAttribute(name)". + * @param name The name of the attribute + * @param value The value of the attribute + */ + public void setAttribute(String name, Object value) { + webAppContext.setAttribute(name, value); + } + + /** + * Add a Jersey resource package. + * @param packageName The Java package name containing the Jersey resource. + * @param pathSpec The path spec for the servlet + */ + public void addJerseyResourcePackage(final String packageName, + final String pathSpec) { + LOG.info("addJerseyResourcePackage: packageName=" + packageName + + ", pathSpec=" + pathSpec); + + ResourceConfig application = new ResourceConfig().packages(packageName); + final ServletHolder sh = new ServletHolder(new ServletContainer(application)); + webAppContext.addServlet(sh, pathSpec); + } + + /** + * Adds a servlet in the server that any user can access. This method differs from + * {@link #addPrivilegedServlet(String, String, Class)} in that any authenticated user + * can interact with the servlet added by this method. + * @param name The name of the servlet (can be passed as null) + * @param pathSpec The path spec for the servlet + * @param clazz The servlet class + */ + public void addUnprivilegedServlet(String name, String pathSpec, + Class clazz) { + addServletWithAuth(name, pathSpec, clazz, false); + } + + /** + * Adds a servlet in the server that only administrators can access. This method differs from + * {@link #addUnprivilegedServlet(String, String, Class)} in that only those authenticated user + * who are identified as administrators can interact with the servlet added by this method. + */ + public void addPrivilegedServlet(String name, String pathSpec, + Class clazz) { + addServletWithAuth(name, pathSpec, clazz, true); + } + + /** + * Internal method to add a servlet to the HTTP server. Developers should not call this method + * directly, but invoke it via {@link #addUnprivilegedServlet(String, String, Class)} or + * {@link #addPrivilegedServlet(String, String, Class)}. + */ + void addServletWithAuth(String name, String pathSpec, + Class clazz, boolean requireAuthz) { + addInternalServlet(name, pathSpec, clazz, requireAuthz); + addFilterPathMapping(pathSpec, webAppContext); + } + + /** + * Add an internal servlet in the server, specifying whether or not to + * protect with Kerberos authentication. + * Note: This method is to be used for adding servlets that facilitate + * internal communication and not for user facing functionality. For + * servlets added using this method, filters (except internal Kerberos + * filters) are not enabled. + * + * @param name The name of the servlet (can be passed as null) + * @param pathSpec The path spec for the servlet + * @param clazz The servlet class + * @param requireAuthz Require Kerberos authenticate to access servlet + */ + void addInternalServlet(String name, String pathSpec, + Class clazz, boolean requireAuthz) { + ServletHolder holder = new ServletHolder(clazz); + if (name != null) { + holder.setName(name); + } + if (authenticationEnabled && requireAuthz) { + FilterHolder filter = new FilterHolder(AdminAuthorizedFilter.class); + filter.setName(AdminAuthorizedFilter.class.getSimpleName()); + FilterMapping fmap = new FilterMapping(); + fmap.setPathSpec(pathSpec); + fmap.setDispatches(FilterMapping.ALL); + fmap.setFilterName(AdminAuthorizedFilter.class.getSimpleName()); + webAppContext.getServletHandler().addFilter(filter, fmap); + } + webAppContext.addServlet(holder, pathSpec); + } + + @Override + public void addFilter(String name, String classname, Map parameters) { + final String[] USER_FACING_URLS = { "*.html", "*.jsp" }; + defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS); + LOG.info("Added filter " + name + " (class=" + classname + + ") to context " + webAppContext.getDisplayName()); + final String[] ALL_URLS = { "/*" }; + for (Map.Entry e : defaultContexts.entrySet()) { + if (e.getValue()) { + ServletContextHandler handler = e.getKey(); + defineFilter(handler, name, classname, parameters, ALL_URLS); + LOG.info("Added filter " + name + " (class=" + classname + + ") to context " + handler.getDisplayName()); + } + } + filterNames.add(name); + } + + @Override + public void addGlobalFilter(String name, String classname, Map parameters) { + final String[] ALL_URLS = { "/*" }; + defineFilter(webAppContext, name, classname, parameters, ALL_URLS); + for (ServletContextHandler ctx : defaultContexts.keySet()) { + defineFilter(ctx, name, classname, parameters, ALL_URLS); + } + LOG.info("Added global filter '" + name + "' (class=" + classname + ")"); + } + + /** + * Define a filter for a context and set up default url mappings. + */ + public static void defineFilter(ServletContextHandler handler, String name, + String classname, Map parameters, String[] urls) { + FilterHolder holder = new FilterHolder(); + holder.setName(name); + holder.setClassName(classname); + if (parameters != null) { + holder.setInitParameters(parameters); + } + FilterMapping fmap = new FilterMapping(); + fmap.setPathSpecs(urls); + fmap.setDispatches(FilterMapping.ALL); + fmap.setFilterName(name); + handler.getServletHandler().addFilter(holder, fmap); + } + + /** + * Add the path spec to the filter path mapping. + * @param pathSpec The path spec + * @param webAppCtx The WebApplicationContext to add to + */ + protected void addFilterPathMapping(String pathSpec, + WebAppContext webAppCtx) { + for(String name : filterNames) { + FilterMapping fmap = new FilterMapping(); + fmap.setPathSpec(pathSpec); + fmap.setFilterName(name); + fmap.setDispatches(FilterMapping.ALL); + webAppCtx.getServletHandler().addFilterMapping(fmap); + } + } + + /** + * Get the value in the webapp context. + * @param name The name of the attribute + * @return The value of the attribute + */ + public Object getAttribute(String name) { + return webAppContext.getAttribute(name); + } + + public WebAppContext getWebAppContext(){ + return this.webAppContext; + } + + public String getWebAppsPath(String appName) throws FileNotFoundException { + return getWebAppsPath(this.appDir, appName); + } + + /** + * Get the pathname to the webapps files. + * @param appName eg "secondary" or "datanode" + * @return the pathname as a URL + * @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH. + */ + protected String getWebAppsPath(String webapps, String appName) throws FileNotFoundException { + URL url = getClass().getClassLoader().getResource(webapps + "/" + appName); + + if (url == null) { + throw new FileNotFoundException(webapps + "/" + appName + + " not found in CLASSPATH"); + } + + String urlString = url.toString(); + return urlString.substring(0, urlString.lastIndexOf('/')); + } + + /** + * Get the port that the server is on + * @return the port + * @deprecated Since 0.99.0 + */ + @Deprecated + public int getPort() { + return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort(); + } + + /** + * Get the address that corresponds to a particular connector. + * + * @return the corresponding address for the connector, or null if there's no + * such connector or the connector is not bounded. + */ + public InetSocketAddress getConnectorAddress(int index) { + Preconditions.checkArgument(index >= 0); + + if (index > webServer.getConnectors().length) { + return null; + } + + ServerConnector c = (ServerConnector)webServer.getConnectors()[index]; + if (c.getLocalPort() == -1 || c.getLocalPort() == -2) { + // -1 if the connector has not been opened + // -2 if it has been closed + return null; + } + + return new InetSocketAddress(c.getHost(), c.getLocalPort()); + } + + /** + * Set the min, max number of worker threads (simultaneous connections). + */ + public void setThreads(int min, int max) { + QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool(); + pool.setMinThreads(min); + pool.setMaxThreads(max); + } + + private void initSpnego(Configuration conf, String hostName, + String usernameConfKey, String keytabConfKey, String kerberosNameRuleKey, + String signatureSecretKeyFileKey) throws IOException { + Map params = new HashMap<>(); + String principalInConf = getOrEmptyString(conf, usernameConfKey); + if (!principalInConf.isEmpty()) { + params.put(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX, SecurityUtil.getServerPrincipal( + principalInConf, hostName)); + } + String httpKeytab = getOrEmptyString(conf, keytabConfKey); + if (!httpKeytab.isEmpty()) { + params.put(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX, httpKeytab); + } + String kerberosNameRule = getOrEmptyString(conf, kerberosNameRuleKey); + if (!kerberosNameRule.isEmpty()) { + params.put(HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX, kerberosNameRule); + } + String signatureSecretKeyFile = getOrEmptyString(conf, signatureSecretKeyFileKey); + if (!signatureSecretKeyFile.isEmpty()) { + params.put(HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX, + signatureSecretKeyFile); + } + params.put(AuthenticationFilter.AUTH_TYPE, "kerberos"); + + // Verify that the required options were provided + if (isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX)) || + isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX))) { + throw new IllegalArgumentException(usernameConfKey + " and " + + keytabConfKey + " are both required in the configuration " + + "to enable SPNEGO/Kerberos authentication for the Web UI"); + } + + if (conf.getBoolean(HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_KEY, + HTTP_SPNEGO_AUTHENTICATION_PROXYUSER_ENABLE_DEFAULT)) { + //Copy/rename standard hadoop proxyuser settings to filter + for(Map.Entry proxyEntry : + conf.getPropsWithPrefix(ProxyUsers.CONF_HADOOP_PROXYUSER).entrySet()) { + params.put(ProxyUserAuthenticationFilter.PROXYUSER_PREFIX + proxyEntry.getKey(), + proxyEntry.getValue()); + } + addGlobalFilter(SPNEGO_PROXYUSER_FILTER, ProxyUserAuthenticationFilter.class.getName(), params); + } else { + addGlobalFilter(SPNEGO_FILTER, AuthenticationFilter.class.getName(), params); + } + } + + /** + * Returns true if the argument is non-null and not whitespace + */ + private boolean isMissing(String value) { + if (null == value) { + return true; + } + return value.trim().isEmpty(); + } + + /** + * Extracts the value for the given key from the configuration of returns a string of + * zero length. + */ + private String getOrEmptyString(Configuration conf, String key) { + if (null == key) { + return EMPTY_STRING; + } + final String value = conf.get(key.trim()); + return null == value ? EMPTY_STRING : value; + } + + /** + * Start the server. Does not wait for the server to start. + */ + public void start() throws IOException { + try { + try { + openListeners(); + webServer.start(); + } catch (IOException ex) { + LOG.info("HttpServer.start() threw a non Bind IOException", ex); + throw ex; + } catch (MultiException ex) { + LOG.info("HttpServer.start() threw a MultiException", ex); + throw ex; + } + // Make sure there is no handler failures. + Handler[] handlers = webServer.getHandlers(); + for (int i = 0; i < handlers.length; i++) { + if (handlers[i].isFailed()) { + throw new IOException( + "Problem in starting http server. Server handlers failed"); + } + } + // Make sure there are no errors initializing the context. + Throwable unavailableException = webAppContext.getUnavailableException(); + if (unavailableException != null) { + // Have to stop the webserver, or else its non-daemon threads + // will hang forever. + webServer.stop(); + throw new IOException("Unable to initialize WebAppContext", + unavailableException); + } + } catch (IOException e) { + throw e; + } catch (InterruptedException e) { + throw (IOException) new InterruptedIOException( + "Interrupted while starting HTTP server").initCause(e); + } catch (Exception e) { + throw new IOException("Problem starting http server", e); + } + } + + private void loadListeners() { + for (ListenerInfo li : listeners) { + webServer.addConnector(li.listener); + } + } + + /** + * Open the main listener for the server + * @throws Exception if the listener cannot be opened or the appropriate port is already in use + */ + @VisibleForTesting + void openListeners() throws Exception { + for (ListenerInfo li : listeners) { + ServerConnector listener = li.listener; + if (!li.isManaged || (li.listener.getLocalPort() != -1 && li.listener.getLocalPort() != -2)) { + // This listener is either started externally, or has not been opened, or has been closed + continue; + } + int port = listener.getPort(); + while (true) { + // jetty has a bug where you can't reopen a listener that previously + // failed to open w/o issuing a close first, even if the port is changed + try { + listener.close(); + listener.open(); + LOG.info("Jetty bound to port " + listener.getLocalPort()); + break; + } catch (IOException ex) { + if(!(ex instanceof BindException) && !(ex.getCause() instanceof BindException)) { + throw ex; + } + if (port == 0 || !findPort) { + BindException be = new BindException("Port in use: " + + listener.getHost() + ":" + listener.getPort()); + be.initCause(ex); + throw be; + } + } + // try the next port number + listener.setPort(++port); + Thread.sleep(100); + } + } + } + + /** + * stop the server + */ + public void stop() throws Exception { + MultiException exception = null; + for (ListenerInfo li : listeners) { + if (!li.isManaged) { + continue; + } + + try { + li.listener.close(); + } catch (Exception e) { + LOG.error( + "Error while stopping listener for webapp" + + webAppContext.getDisplayName(), e); + exception = addMultiException(exception, e); + } + } + + try { + // clear & stop webAppContext attributes to avoid memory leaks. + webAppContext.clearAttributes(); + webAppContext.stop(); + } catch (Exception e) { + LOG.error("Error while stopping web app context for webapp " + + webAppContext.getDisplayName(), e); + exception = addMultiException(exception, e); + } + + try { + webServer.stop(); + } catch (Exception e) { + LOG.error("Error while stopping web server for webapp " + + webAppContext.getDisplayName(), e); + exception = addMultiException(exception, e); + } + + if (exception != null) { + exception.ifExceptionThrow(); + } + + } + + private MultiException addMultiException(MultiException exception, Exception e) { + if(exception == null){ + exception = new MultiException(); + } + exception.add(e); + return exception; + } + + public void join() throws InterruptedException { + webServer.join(); + } + + /** + * Test for the availability of the web server + * @return true if the web server is started, false otherwise + */ + public boolean isAlive() { + return webServer != null && webServer.isStarted(); + } + + /** + * Return the host and port of the HttpServer, if live + * @return the classname and any HTTP URL + */ + @Override + public String toString() { + if (listeners.isEmpty()) { + return "Inactive HttpServer"; + } else { + StringBuilder sb = new StringBuilder("HttpServer (") + .append(isAlive() ? STATE_DESCRIPTION_ALIVE : + STATE_DESCRIPTION_NOT_LIVE).append("), listening at:"); + for (ListenerInfo li : listeners) { + ServerConnector l = li.listener; + sb.append(l.getHost()).append(":").append(l.getPort()).append("/,"); + } + return sb.toString(); + } + } + + /** + * Checks the user has privileges to access to instrumentation servlets. + *

+ * If hadoop.security.instrumentation.requires.admin is set to FALSE + * (default value) it always returns TRUE. + *

+ * If hadoop.security.instrumentation.requires.admin is set to TRUE + * it will check that if the current user is in the admin ACLS. If the user is + * in the admin ACLs it returns TRUE, otherwise it returns FALSE. + *

+ * + * @param servletContext the servlet context. + * @param request the servlet request. + * @param response the servlet response. + * @return TRUE/FALSE based on the logic decribed above. + */ + public static boolean isInstrumentationAccessAllowed( + ServletContext servletContext, HttpServletRequest request, + HttpServletResponse response) throws IOException { + Configuration conf = + (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE); + + boolean access = true; + boolean adminAccess = conf.getBoolean( + CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN, + false); + if (adminAccess) { + access = hasAdministratorAccess(servletContext, request, response); + } + return access; + } + + /** + * Does the user sending the HttpServletRequest has the administrator ACLs? If + * it isn't the case, response will be modified to send an error to the user. + * + * @param servletContext the {@link ServletContext} to use + * @param request the {@link HttpServletRequest} to check + * @param response used to send the error response if user does not have admin access. + * @return true if admin-authorized, false otherwise + * @throws IOException if an unauthenticated or unauthorized user tries to access the page + */ + public static boolean hasAdministratorAccess( + ServletContext servletContext, HttpServletRequest request, + HttpServletResponse response) throws IOException { + Configuration conf = + (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE); + AccessControlList acl = (AccessControlList) servletContext.getAttribute(ADMINS_ACL); + + return hasAdministratorAccess(conf, acl, request, response); + } + + public static boolean hasAdministratorAccess(Configuration conf, AccessControlList acl, + HttpServletRequest request, HttpServletResponse response) throws IOException { + // If there is no authorization, anybody has administrator access. + if (!conf.getBoolean( + CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) { + return true; + } + + String remoteUser = request.getRemoteUser(); + if (remoteUser == null) { + response.sendError(HttpServletResponse.SC_UNAUTHORIZED, + "Unauthenticated users are not " + + "authorized to access this page."); + return false; + } + + if (acl != null && !userHasAdministratorAccess(acl, remoteUser)) { + response.sendError(HttpServletResponse.SC_FORBIDDEN, "User " + + remoteUser + " is unauthorized to access this page."); + return false; + } + + return true; + } + + /** + * Get the admin ACLs from the given ServletContext and check if the given + * user is in the ACL. + * + * @param servletContext the context containing the admin ACL. + * @param remoteUser the remote user to check for. + * @return true if the user is present in the ACL, false if no ACL is set or + * the user is not present + */ + public static boolean userHasAdministratorAccess(ServletContext servletContext, + String remoteUser) { + AccessControlList adminsAcl = (AccessControlList) servletContext + .getAttribute(ADMINS_ACL); + return userHasAdministratorAccess(adminsAcl, remoteUser); + } + + public static boolean userHasAdministratorAccess(AccessControlList acl, String remoteUser) { + UserGroupInformation remoteUserUGI = + UserGroupInformation.createRemoteUser(remoteUser); + return acl != null && acl.isUserAllowed(remoteUserUGI); + } + + /** + * A very simple servlet to serve up a text representation of the current + * stack traces. It both returns the stacks to the caller and logs them. + * Currently the stack traces are done sequentially rather than exactly the + * same data. + */ + public static class StackServlet extends HttpServlet { + private static final long serialVersionUID = -6284183679759467039L; + + @Override + public void doGet(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), + request, response)) { + return; + } + response.setContentType("text/plain; charset=UTF-8"); + try (PrintStream out = new PrintStream( + response.getOutputStream(), false, "UTF-8")) { + Threads.printThreadInfo(out, ""); + out.flush(); + } + ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1); + } + } + + /** + * A Servlet input filter that quotes all HTML active characters in the + * parameter names and values. The goal is to quote the characters to make + * all of the servlets resistant to cross-site scripting attacks. + */ + @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) + public static class QuotingInputFilter implements Filter { + private FilterConfig config; + + public static class RequestQuoter extends HttpServletRequestWrapper { + private final HttpServletRequest rawRequest; + public RequestQuoter(HttpServletRequest rawRequest) { + super(rawRequest); + this.rawRequest = rawRequest; + } + + /** + * Return the set of parameter names, quoting each name. + */ + @Override + public Enumeration getParameterNames() { + return new Enumeration() { + private Enumeration rawIterator = + rawRequest.getParameterNames(); + @Override + public boolean hasMoreElements() { + return rawIterator.hasMoreElements(); + } + + @Override + public String nextElement() { + return HtmlQuoting.quoteHtmlChars(rawIterator.nextElement()); + } + }; + } + + /** + * Unquote the name and quote the value. + */ + @Override + public String getParameter(String name) { + return HtmlQuoting.quoteHtmlChars(rawRequest.getParameter( + HtmlQuoting.unquoteHtmlChars(name))); + } + + @Override + public String[] getParameterValues(String name) { + String unquoteName = HtmlQuoting.unquoteHtmlChars(name); + String[] unquoteValue = rawRequest.getParameterValues(unquoteName); + if (unquoteValue == null) { + return null; + } + String[] result = new String[unquoteValue.length]; + for(int i=0; i < result.length; ++i) { + result[i] = HtmlQuoting.quoteHtmlChars(unquoteValue[i]); + } + return result; + } + + @Override + public Map getParameterMap() { + Map result = new HashMap<>(); + Map raw = rawRequest.getParameterMap(); + for (Map.Entry item: raw.entrySet()) { + String[] rawValue = item.getValue(); + String[] cookedValue = new String[rawValue.length]; + for(int i=0; i< rawValue.length; ++i) { + cookedValue[i] = HtmlQuoting.quoteHtmlChars(rawValue[i]); + } + result.put(HtmlQuoting.quoteHtmlChars(item.getKey()), cookedValue); + } + return result; + } + + /** + * Quote the url so that users specifying the HOST HTTP header + * can't inject attacks. + */ + @Override + public StringBuffer getRequestURL(){ + String url = rawRequest.getRequestURL().toString(); + return new StringBuffer(HtmlQuoting.quoteHtmlChars(url)); + } + + /** + * Quote the server name so that users specifying the HOST HTTP header + * can't inject attacks. + */ + @Override + public String getServerName() { + return HtmlQuoting.quoteHtmlChars(rawRequest.getServerName()); + } + } + + @Override + public void init(FilterConfig config) throws ServletException { + this.config = config; + } + + @Override + public void destroy() { + } + + @Override + public void doFilter(ServletRequest request, + ServletResponse response, + FilterChain chain + ) throws IOException, ServletException { + HttpServletRequestWrapper quoted = + new RequestQuoter((HttpServletRequest) request); + HttpServletResponse httpResponse = (HttpServletResponse) response; + + String mime = inferMimeType(request); + if (mime == null) { + httpResponse.setContentType("text/plain; charset=utf-8"); + } else if (mime.startsWith("text/html")) { + // HTML with unspecified encoding, we want to + // force HTML with utf-8 encoding + // This is to avoid the following security issue: + // http://openmya.hacker.jp/hasegawa/security/utf7cs.html + httpResponse.setContentType("text/html; charset=utf-8"); + } else if (mime.startsWith("application/xml")) { + httpResponse.setContentType("text/xml; charset=utf-8"); + } + chain.doFilter(quoted, httpResponse); + } + + /** + * Infer the mime type for the response based on the extension of the request + * URI. Returns null if unknown. + */ + private String inferMimeType(ServletRequest request) { + String path = ((HttpServletRequest)request).getRequestURI(); + ServletContext context = config.getServletContext(); + return context.getMimeType(path); + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServerUtil.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServerUtil.java new file mode 100644 index 00000000..a043ba6c --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/HttpServerUtil.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.yetus.audience.InterfaceAudience; +import org.eclipse.jetty.security.ConstraintMapping; +import org.eclipse.jetty.security.ConstraintSecurityHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.util.security.Constraint; + +/** + * HttpServer utility. + */ +@InterfaceAudience.Private +public final class HttpServerUtil { + /** + * Add constraints to a Jetty Context to disallow undesirable Http methods. + * @param ctxHandler The context to modify + * @param allowOptionsMethod if true then OPTIONS method will not be set in constraint mapping + */ + public static void constrainHttpMethods(ServletContextHandler ctxHandler, + boolean allowOptionsMethod) { + Constraint c = new Constraint(); + c.setAuthenticate(true); + + ConstraintMapping cmt = new ConstraintMapping(); + cmt.setConstraint(c); + cmt.setMethod("TRACE"); + cmt.setPathSpec("/*"); + + ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler(); + + if (!allowOptionsMethod) { + ConstraintMapping cmo = new ConstraintMapping(); + cmo.setConstraint(c); + cmo.setMethod("OPTIONS"); + cmo.setPathSpec("/*"); + securityHandler.setConstraintMappings(new ConstraintMapping[] { cmt, cmo }); + } else { + securityHandler.setConstraintMappings(new ConstraintMapping[] { cmt }); + } + + ctxHandler.setSecurityHandler(securityHandler); + } + + private HttpServerUtil() {} +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/InfoServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/InfoServer.java new file mode 100644 index 00000000..33079740 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/InfoServer.java @@ -0,0 +1,182 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.yetus.audience.InterfaceAudience; + +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import java.io.IOException; +import java.net.URI; + +/** + * Create a Jetty embedded server to answer http requests. The primary goal + * is to serve up status information for the server. + * There are three contexts: + * "/stacks/" -> points to stack trace + * "/static/" -> points to common static files (src/hbase-webapps/static) + * "/" -> the jsp server code from (src/hbase-webapps/<name>) + */ +@InterfaceAudience.Private +public class InfoServer { + private static final String HBASE_APP_DIR = "hbase-webapps"; + private final HttpServer httpServer; + + /** + * Create a status server on the given port. + * The jsp scripts are taken from src/hbase-webapps/name. + * @param name The name of the server + * @param bindAddress address to bind to + * @param port The port to use on the server + * @param findPort whether the server should start at the given port and increment by 1 until it + * finds a free port. + * @param c the {@link Configuration} to build the server + * @throws IOException if getting one of the password fails or the server cannot be created + */ + public InfoServer(String name, String bindAddress, int port, boolean findPort, + final Configuration c) throws IOException { + HttpConfig httpConfig = new HttpConfig(c); + HttpServer.Builder builder = + new HttpServer.Builder(); + + builder.setName(name).addEndpoint(URI.create(httpConfig.getSchemePrefix() + + bindAddress + ":" + + port)).setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c); + String logDir = System.getProperty("hbase.log.dir"); + if (logDir != null) { + builder.setLogDir(logDir); + } + if (httpConfig.isSecure()) { + builder.keyPassword(HBaseConfiguration + .getPassword(c, "ssl.server.keystore.keypassword", null)) + .keyStore(c.get("ssl.server.keystore.location"), + HBaseConfiguration.getPassword(c,"ssl.server.keystore.password", null), + c.get("ssl.server.keystore.type", "jks")) + .trustStore(c.get("ssl.server.truststore.location"), + HBaseConfiguration.getPassword(c, "ssl.server.truststore.password", null), + c.get("ssl.server.truststore.type", "jks")); + } + // Enable SPNEGO authentication + if ("kerberos".equalsIgnoreCase(c.get(HttpServer.HTTP_UI_AUTHENTICATION, null))) { + builder.setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY) + .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY) + .setKerberosNameRulesKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY) + .setSignatureSecretFileKey( + HttpServer.HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY) + .setSecurityEnabled(true); + + // Set an admin ACL on sensitive webUI endpoints + AccessControlList acl = buildAdminAcl(c); + builder.setACL(acl); + } + this.httpServer = builder.build(); + } + + /** + * Builds an ACL that will restrict the users who can issue commands to endpoints on the UI + * which are meant only for administrators. + */ + AccessControlList buildAdminAcl(Configuration conf) { + final String userGroups = conf.get(HttpServer.HTTP_SPNEGO_AUTHENTICATION_ADMIN_USERS_KEY, null); + final String adminGroups = conf.get( + HttpServer.HTTP_SPNEGO_AUTHENTICATION_ADMIN_GROUPS_KEY, null); + if (userGroups == null && adminGroups == null) { + // Backwards compatibility - if the user doesn't have anything set, allow all users in. + return new AccessControlList("*", null); + } + return new AccessControlList(userGroups, adminGroups); + } + + /** + * Explicitly invoke {@link #addPrivilegedServlet(String, String, Class)} or + * {@link #addUnprivilegedServlet(String, String, Class)} instead of this method. + * This method will add a servlet which any authenticated user can access. + * + * @deprecated Use {@link #addUnprivilegedServlet(String, String, Class)} or + * {@link #addPrivilegedServlet(String, String, Class)} instead of this + * method which does not state outwardly what kind of authz rules will + * be applied to this servlet. + */ + @Deprecated + public void addServlet(String name, String pathSpec, + Class clazz) { + addUnprivilegedServlet(name, pathSpec, clazz); + } + + /** + * @see HttpServer#addUnprivilegedServlet(String, String, Class) + */ + public void addUnprivilegedServlet(String name, String pathSpec, + Class clazz) { + this.httpServer.addUnprivilegedServlet(name, pathSpec, clazz); + } + + /** + * @see HttpServer#addPrivilegedServlet(String, String, Class) + */ + public void addPrivilegedServlet(String name, String pathSpec, + Class clazz) { + this.httpServer.addPrivilegedServlet(name, pathSpec, clazz); + } + + public void setAttribute(String name, Object value) { + this.httpServer.setAttribute(name, value); + } + + public void start() throws IOException { + this.httpServer.start(); + } + + /** + * @return the port of the info server + * @deprecated Since 0.99.0 + */ + @Deprecated + public int getPort() { + return this.httpServer.getPort(); + } + + public void stop() throws Exception { + this.httpServer.stop(); + } + + + /** + * Returns true if and only if UI authentication (spnego) is enabled, UI authorization is enabled, + * and the requesting user is defined as an administrator. If the UI is set to readonly, this + * method always returns false. + */ + public static boolean canUserModifyUI( + HttpServletRequest req, ServletContext ctx, Configuration conf) { + if (conf.getBoolean("hbase.master.ui.readonly", false)) { + return false; + } + String remoteUser = req.getRemoteUser(); + if ("kerberos".equalsIgnoreCase(conf.get(HttpServer.HTTP_UI_AUTHENTICATION)) && + conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false) && + remoteUser != null) { + return HttpServer.userHasAdministratorAccess(ctx, remoteUser); + } + return false; + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileOutputServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileOutputServlet.java new file mode 100644 index 00000000..c867b727 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileOutputServlet.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.yetus.audience.InterfaceAudience; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.util.regex.Pattern; + +/** + * Servlet to serve files generated by {@link ProfileServlet} + */ +@InterfaceAudience.Private +public class ProfileOutputServlet extends DefaultServlet { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ProfileOutputServlet.class); + private static final int REFRESH_PERIOD = 2; + // Alphanumeric characters, plus percent (url-encoding), equals, ampersand, dot and hyphen + private static final Pattern ALPHA_NUMERIC = Pattern.compile("[a-zA-Z0-9%=&.\\-]*"); + + @Override + protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) + throws ServletException, IOException { + String absoluteDiskPath = getServletContext().getRealPath(req.getPathInfo()); + File requestedFile = new File(absoluteDiskPath); + // async-profiler version 1.4 writes 'Started [cpu] profiling' to output file when profiler is + // running which gets replaced by final output. If final output is not ready yet, the file size + // will be <100 bytes (in all modes). + if (requestedFile.length() < 100) { + LOG.info(requestedFile + " is incomplete. Sending auto-refresh header."); + String refreshUrl = req.getRequestURI(); + // Rebuild the query string (if we have one) + if (req.getQueryString() != null) { + refreshUrl += "?" + sanitize(req.getQueryString()); + } + ProfileServlet.setResponseHeader(resp); + resp.setHeader("Refresh", REFRESH_PERIOD + ";" + refreshUrl); + resp.getWriter().write("This page will be auto-refreshed every " + REFRESH_PERIOD + + " seconds until the output file is ready. Redirecting to " + refreshUrl); + } else { + super.doGet(req, resp); + } + } + + static String sanitize(String input) { + // Basic test to try to avoid any XSS attacks or HTML content showing up. + // Duplicates HtmlQuoting a little, but avoid destroying ampersand. + if (ALPHA_NUMERIC.matcher(input).matches()) { + return input; + } + throw new RuntimeException("Non-alphanumeric data found in input, aborting."); + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileServlet.java new file mode 100644 index 00000000..1cb3d68a --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProfileServlet.java @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.hbase.rest.util.ProcessUtils; +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Servlet that runs async-profiler as web-endpoint. + * Following options from async-profiler can be specified as query paramater. + * // -e event profiling event: cpu|alloc|lock|cache-misses etc. + * // -d duration run profiling for 'duration' seconds (integer) + * // -i interval sampling interval in nanoseconds (long) + * // -j jstackdepth maximum Java stack depth (integer) + * // -b bufsize frame buffer size (long) + * // -t profile different threads separately + * // -s simple class names instead of FQN + * // -o fmt[,fmt...] output format: summary|traces|flat|collapsed|svg|tree|jfr + * // --width px SVG width pixels (integer) + * // --height px SVG frame height pixels (integer) + * // --minwidth px skip frames smaller than px (double) + * // --reverse generate stack-reversed FlameGraph / Call tree + * Example: + * - To collect 30 second CPU profile of current process (returns FlameGraph svg) + * curl "http://localhost:10002/prof" + * - To collect 1 minute CPU profile of current process and output in tree format (html) + * curl "http://localhost:10002/prof?output=tree&duration=60" + * - To collect 30 second heap allocation profile of current process (returns FlameGraph svg) + * curl "http://localhost:10002/prof?event=alloc" + * - To collect lock contention profile of current process (returns FlameGraph svg) + * curl "http://localhost:10002/prof?event=lock" + * Following event types are supported (default is 'cpu') (NOTE: not all OS'es support all events) + * // Perf events: + * // cpu + * // page-faults + * // context-switches + * // cycles + * // instructions + * // cache-references + * // cache-misses + * // branches + * // branch-misses + * // bus-cycles + * // L1-dcache-load-misses + * // LLC-load-misses + * // dTLB-load-misses + * // mem:breakpoint + * // trace:tracepoint + * // Java events: + * // alloc + * // lock + */ +@InterfaceAudience.Private +public class ProfileServlet extends HttpServlet { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ProfileServlet.class); + + private static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods"; + private static final String ALLOWED_METHODS = "GET"; + private static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin"; + private static final String CONTENT_TYPE_TEXT = "text/plain; charset=utf-8"; + private static final String ASYNC_PROFILER_HOME_ENV = "ASYNC_PROFILER_HOME"; + private static final String ASYNC_PROFILER_HOME_SYSTEM_PROPERTY = "async.profiler.home"; + private static final String PROFILER_SCRIPT = "/profiler.sh"; + private static final int DEFAULT_DURATION_SECONDS = 10; + private static final AtomicInteger ID_GEN = new AtomicInteger(0); + static final String OUTPUT_DIR = System.getProperty("java.io.tmpdir") + "/prof-output"; + + enum Event { + CPU("cpu"), + ALLOC("alloc"), + LOCK("lock"), + PAGE_FAULTS("page-faults"), + CONTEXT_SWITCHES("context-switches"), + CYCLES("cycles"), + INSTRUCTIONS("instructions"), + CACHE_REFERENCES("cache-references"), + CACHE_MISSES("cache-misses"), + BRANCHES("branches"), + BRANCH_MISSES("branch-misses"), + BUS_CYCLES("bus-cycles"), + L1_DCACHE_LOAD_MISSES("L1-dcache-load-misses"), + LLC_LOAD_MISSES("LLC-load-misses"), + DTLB_LOAD_MISSES("dTLB-load-misses"), + MEM_BREAKPOINT("mem:breakpoint"), + TRACE_TRACEPOINT("trace:tracepoint"),; + + private final String internalName; + + Event(final String internalName) { + this.internalName = internalName; + } + + public String getInternalName() { + return internalName; + } + + public static Event fromInternalName(final String name) { + for (Event event : values()) { + if (event.getInternalName().equalsIgnoreCase(name)) { + return event; + } + } + + return null; + } + } + + enum Output { + SUMMARY, + TRACES, + FLAT, + COLLAPSED, + SVG, + TREE, + JFR + } + + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED", + justification = "This class is never serialized nor restored.") + private transient Lock profilerLock = new ReentrantLock(); + private transient volatile Process process; + private String asyncProfilerHome; + private Integer pid; + + public ProfileServlet() { + this.asyncProfilerHome = getAsyncProfilerHome(); + this.pid = ProcessUtils.getPid(); + LOG.info("Servlet process PID: " + pid + " asyncProfilerHome: " + asyncProfilerHome); + } + + @Override + protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) + throws IOException { + if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), req, resp)) { + resp.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + setResponseHeader(resp); + resp.getWriter().write("Unauthorized: Instrumentation access is not allowed!"); + return; + } + + // make sure async profiler home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(resp); + resp.getWriter().write("ASYNC_PROFILER_HOME env is not set.\n\n" + + "Please ensure the prerequsites for the Profiler Servlet have been installed and the\n" + + "environment is properly configured. For more information please see\n" + + "http://hbase.apache.org/book.html#profiler\n"); + return; + } + + // if pid is explicitly specified, use it else default to current process + pid = getInteger(req, "pid", pid); + + // if pid is not specified in query param and if current process pid cannot be determined + if (pid == null) { + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(resp); + resp.getWriter().write( + "'pid' query parameter unspecified or unable to determine PID of current process."); + return; + } + + final int duration = getInteger(req, "duration", DEFAULT_DURATION_SECONDS); + final Output output = getOutput(req); + final Event event = getEvent(req); + final Long interval = getLong(req, "interval"); + final Integer jstackDepth = getInteger(req, "jstackdepth", null); + final Long bufsize = getLong(req, "bufsize"); + final boolean thread = req.getParameterMap().containsKey("thread"); + final boolean simple = req.getParameterMap().containsKey("simple"); + final Integer width = getInteger(req, "width", null); + final Integer height = getInteger(req, "height", null); + final Double minwidth = getMinWidth(req); + final boolean reverse = req.getParameterMap().containsKey("reverse"); + + if (process == null || !process.isAlive()) { + try { + int lockTimeoutSecs = 3; + if (profilerLock.tryLock(lockTimeoutSecs, TimeUnit.SECONDS)) { + try { + File outputFile = new File(OUTPUT_DIR, "async-prof-pid-" + pid + "-" + + event.name().toLowerCase() + "-" + ID_GEN.incrementAndGet() + "." + + output.name().toLowerCase()); + List cmd = new ArrayList<>(); + cmd.add(asyncProfilerHome + PROFILER_SCRIPT); + cmd.add("-e"); + cmd.add(event.getInternalName()); + cmd.add("-d"); + cmd.add("" + duration); + cmd.add("-o"); + cmd.add(output.name().toLowerCase()); + cmd.add("-f"); + cmd.add(outputFile.getAbsolutePath()); + if (interval != null) { + cmd.add("-i"); + cmd.add(interval.toString()); + } + if (jstackDepth != null) { + cmd.add("-j"); + cmd.add(jstackDepth.toString()); + } + if (bufsize != null) { + cmd.add("-b"); + cmd.add(bufsize.toString()); + } + if (thread) { + cmd.add("-t"); + } + if (simple) { + cmd.add("-s"); + } + if (width != null) { + cmd.add("--width"); + cmd.add(width.toString()); + } + if (height != null) { + cmd.add("--height"); + cmd.add(height.toString()); + } + if (minwidth != null) { + cmd.add("--minwidth"); + cmd.add(minwidth.toString()); + } + if (reverse) { + cmd.add("--reverse"); + } + cmd.add(pid.toString()); + process = ProcessUtils.runCmdAsync(cmd); + + // set response and set refresh header to output location + setResponseHeader(resp); + resp.setStatus(HttpServletResponse.SC_ACCEPTED); + String relativeUrl = "/prof-output/" + outputFile.getName(); + resp.getWriter().write( + "Started [" + event.getInternalName() + + "] profiling. This page will automatically redirect to " + + relativeUrl + " after " + duration + " seconds. " + + "If empty diagram and Linux 4.6+, see 'Basic Usage' section on the Async " + + "Profiler Home Page, https://github.com/jvm-profiling-tools/async-profiler." + + "\n\nCommand:\n" + + Joiner.on(" ").join(cmd)); + + // to avoid auto-refresh by ProfileOutputServlet, refreshDelay can be specified + // via url param + int refreshDelay = getInteger(req, "refreshDelay", 0); + + // instead of sending redirect, set auto-refresh so that browsers will refresh + // with redirected url + resp.setHeader("Refresh", (duration + refreshDelay) + ";" + relativeUrl); + resp.getWriter().flush(); + } finally { + profilerLock.unlock(); + } + } else { + setResponseHeader(resp); + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + resp.getWriter().write( + "Unable to acquire lock. Another instance of profiler might be running."); + LOG.warn("Unable to acquire lock in " + lockTimeoutSecs + + " seconds. Another instance of profiler might be running."); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while acquiring profile lock.", e); + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } + } else { + setResponseHeader(resp); + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + resp.getWriter().write("Another instance of profiler is already running."); + } + } + + private Integer getInteger(final HttpServletRequest req, final String param, + final Integer defaultValue) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Integer.valueOf(value); + } catch (NumberFormatException e) { + return defaultValue; + } + } + return defaultValue; + } + + private Long getLong(final HttpServletRequest req, final String param) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Long.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Double getMinWidth(final HttpServletRequest req) { + final String value = req.getParameter("minwidth"); + if (value != null) { + try { + return Double.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Event getEvent(final HttpServletRequest req) { + final String eventArg = req.getParameter("event"); + if (eventArg != null) { + Event event = Event.fromInternalName(eventArg); + return event == null ? Event.CPU : event; + } + return Event.CPU; + } + + private Output getOutput(final HttpServletRequest req) { + final String outputArg = req.getParameter("output"); + if (req.getParameter("output") != null) { + try { + return Output.valueOf(outputArg.trim().toUpperCase()); + } catch (IllegalArgumentException e) { + return Output.SVG; + } + } + return Output.SVG; + } + + static void setResponseHeader(final HttpServletResponse response) { + response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS); + response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*"); + response.setContentType(CONTENT_TYPE_TEXT); + } + + static String getAsyncProfilerHome() { + String asyncProfilerHome = System.getenv(ASYNC_PROFILER_HOME_ENV); + // if ENV is not set, see if -Dasync.profiler.home=/path/to/async/profiler/home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + asyncProfilerHome = System.getProperty(ASYNC_PROFILER_HOME_SYSTEM_PROPERTY); + } + + return asyncProfilerHome; + } + + public static class DisabledServlet extends HttpServlet { + + private static final long serialVersionUID = 1L; + + @Override + protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) + throws IOException { + resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(resp); + resp.getWriter().write("The profiler servlet was disabled at startup.\n\n" + + "Please ensure the prerequsites for the Profiler Servlet have been installed and the\n" + + "environment is properly configured. For more information please see\n" + + "http://hbase.apache.org/book.html#profiler\n"); + return; + } + + } + +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProxyUserAuthenticationFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProxyUserAuthenticationFilter.java new file mode 100644 index 00000000..55e774a2 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ProxyUserAuthenticationFilter.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.server.AuthenticationFilter; +import org.apache.hadoop.security.authorize.AuthorizationException; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hadoop.util.HttpExceptionUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletRequestWrapper; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.security.Principal; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * This file has been copied directly (changing only the package name and and the ASF license + * text format, and adding the Yetus annotations) from Hadoop, as the Hadoop version that HBase + * depends on doesn't have it yet + * (as of 2020 Apr 24, there is no Hadoop release that has it either). + * + * Hadoop version: + * unreleased, master branch commit 4ea6c2f457496461afc63f38ef4cef3ab0efce49 + * + * Haddop path: + * hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authentication/ + * server/ProxyUserAuthenticationFilter.java + * + * AuthenticationFilter which adds support to perform operations + * using end user instead of proxy user. Fetches the end user from + * doAs Query Parameter. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class ProxyUserAuthenticationFilter extends AuthenticationFilter { + + private static final Logger LOG = LoggerFactory.getLogger( + ProxyUserAuthenticationFilter.class); + + private static final String DO_AS = "doas"; + public static final String PROXYUSER_PREFIX = "proxyuser"; + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + Configuration conf = getProxyuserConfiguration(filterConfig); + ProxyUsers.refreshSuperUserGroupsConfiguration(conf, PROXYUSER_PREFIX); + super.init(filterConfig); + } + + @Override + protected void doFilter(FilterChain filterChain, HttpServletRequest request, + HttpServletResponse response) throws IOException, ServletException { + final HttpServletRequest lowerCaseRequest = toLowerCase(request); + String doAsUser = lowerCaseRequest.getParameter(DO_AS); + + if (doAsUser != null && !doAsUser.equals(request.getRemoteUser())) { + LOG.debug("doAsUser = {}, RemoteUser = {} , RemoteAddress = {} ", + doAsUser, request.getRemoteUser(), request.getRemoteAddr()); + UserGroupInformation requestUgi = (request.getUserPrincipal() != null) ? + UserGroupInformation.createRemoteUser(request.getRemoteUser()) + : null; + if (requestUgi != null) { + requestUgi = UserGroupInformation.createProxyUser(doAsUser, + requestUgi); + try { + ProxyUsers.authorize(requestUgi, request.getRemoteAddr()); + + final UserGroupInformation ugiF = requestUgi; + request = new HttpServletRequestWrapper(request) { + @Override + public String getRemoteUser() { + return ugiF.getShortUserName(); + } + + @Override + public Principal getUserPrincipal() { + return new Principal() { + @Override + public String getName() { + return ugiF.getUserName(); + } + }; + } + }; + LOG.debug("Proxy user Authentication successful"); + } catch (AuthorizationException ex) { + HttpExceptionUtils.createServletExceptionResponse(response, + HttpServletResponse.SC_FORBIDDEN, ex); + LOG.warn("Proxy user Authentication exception", ex); + return; + } + } + } + super.doFilter(filterChain, request, response); + } + + protected Configuration getProxyuserConfiguration(FilterConfig filterConfig) + throws ServletException { + Configuration conf = new Configuration(false); + Enumeration names = filterConfig.getInitParameterNames(); + while (names.hasMoreElements()) { + String name = (String) names.nextElement(); + if (name.startsWith(PROXYUSER_PREFIX + ".")) { + String value = filterConfig.getInitParameter(name); + conf.set(name, value); + } + } + return conf; + } + + static boolean containsUpperCase(final Iterable strings) { + for(String s : strings) { + for(int i = 0; i < s.length(); i++) { + if (Character.isUpperCase(s.charAt(i))) { + return true; + } + } + } + return false; + } + + public static HttpServletRequest toLowerCase( + final HttpServletRequest request) { + @SuppressWarnings("unchecked") + final Map original = (Map) + request.getParameterMap(); + if (!containsUpperCase(original.keySet())) { + return request; + } + + final Map> m = new HashMap>(); + for (Map.Entry entry : original.entrySet()) { + final String key = StringUtils.toLowerCase(entry.getKey()); + List strings = m.get(key); + if (strings == null) { + strings = new ArrayList(); + m.put(key, strings); + } + for (String v : entry.getValue()) { + strings.add(v); + } + } + + return new HttpServletRequestWrapper(request) { + private Map parameters = null; + + @Override + public Map getParameterMap() { + if (parameters == null) { + parameters = new HashMap(); + for (Map.Entry> entry : m.entrySet()) { + final List a = entry.getValue(); + parameters.put(entry.getKey(), a.toArray(new String[a.size()])); + } + } + return parameters; + } + + @Override + public String getParameter(String name) { + final List a = m.get(name); + return a == null ? null : a.get(0); + } + + @Override + public String[] getParameterValues(String name) { + return getParameterMap().get(name); + } + + @Override + public Enumeration getParameterNames() { + final Iterator i = m.keySet().iterator(); + return new Enumeration() { + @Override + public boolean hasMoreElements() { + return i.hasNext(); + } + + @Override + public String nextElement() { + return i.next(); + } + }; + } + }; + } + +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/SecurityHeadersFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/SecurityHeadersFilter.java new file mode 100644 index 00000000..b2cddc0c --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/SecurityHeadersFilter.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.rest.http; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) +public class SecurityHeadersFilter implements Filter { + private static final Logger LOG = + LoggerFactory.getLogger(SecurityHeadersFilter.class); + private static final String DEFAULT_HSTS = ""; + private static final String DEFAULT_CSP = ""; + private FilterConfig filterConfig; + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + this.filterConfig = filterConfig; + LOG.info("Added security headers filter"); + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws IOException, ServletException { + HttpServletResponse httpResponse = (HttpServletResponse) response; + httpResponse.addHeader("X-Content-Type-Options", "nosniff"); + httpResponse.addHeader("X-XSS-Protection", "1; mode=block"); + String hsts = filterConfig.getInitParameter("hsts"); + if (StringUtils.isNotBlank(hsts)) { + httpResponse.addHeader("Strict-Transport-Security", hsts); + } + String csp = filterConfig.getInitParameter("csp"); + if (StringUtils.isNotBlank(csp)) { + httpResponse.addHeader("Content-Security-Policy", csp); + } + chain.doFilter(request, response); + } + + @Override + public void destroy() { + } + + public static Map getDefaultParameters(Configuration conf) { + Map params = new HashMap<>(); + params.put("hsts", conf.get("hbase.http.filter.hsts.value", + DEFAULT_HSTS)); + params.put("csp", conf.get("hbase.http.filter.csp.value", + DEFAULT_CSP)); + return params; + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ServerConfigurationKeys.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ServerConfigurationKeys.java new file mode 100644 index 00000000..8b8f9aeb --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/ServerConfigurationKeys.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +/** + * This interface contains constants for configuration keys used + * in the hbase http server code. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface ServerConfigurationKeys { + + /** Enable/Disable ssl for http server */ + public static final String HBASE_SSL_ENABLED_KEY = "hbase.ssl.enabled"; + + public static final boolean HBASE_SSL_ENABLED_DEFAULT = false; + + /** Enable/Disable aliases serving from jetty */ + public static final String HBASE_JETTY_LOGS_SERVE_ALIASES = + "hbase.jetty.logs.serve.aliases"; + + public static final boolean DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES = + true; + + public static final String HBASE_HTTP_STATIC_USER = "hbase.http.staticuser.user"; + + public static final String DEFAULT_HBASE_HTTP_STATIC_USER = "dr.stack"; + +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/conf/ConfServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/conf/ConfServlet.java new file mode 100644 index 00000000..f3c69f3b --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/conf/ConfServlet.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http.conf; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.rest.http.HttpServer; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.io.Writer; + +/** + * A servlet to print out the running configuration data. + */ +@InterfaceAudience.LimitedPrivate({"HBase"}) +@InterfaceStability.Unstable +public class ConfServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + + private static final String FORMAT_JSON = "json"; + private static final String FORMAT_XML = "xml"; + private static final String FORMAT_PARAM = "format"; + + /** + * Return the Configuration of the daemon hosting this servlet. + * This is populated when the HttpServer starts. + */ + private Configuration getConfFromContext() { + Configuration conf = (Configuration)getServletContext().getAttribute( + HttpServer.CONF_CONTEXT_ATTRIBUTE); + assert conf != null; + return conf; + } + + @Override + public void doGet(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), + request, response)) { + return; + } + + String format = request.getParameter(FORMAT_PARAM); + if (null == format) { + format = FORMAT_XML; + } + + if (FORMAT_XML.equals(format)) { + response.setContentType("text/xml; charset=utf-8"); + } else if (FORMAT_JSON.equals(format)) { + response.setContentType("application/json; charset=utf-8"); + } + + Writer out = response.getWriter(); + try { + writeResponse(getConfFromContext(), out, format); + } catch (BadFormatException bfe) { + response.sendError(HttpServletResponse.SC_BAD_REQUEST, bfe.getMessage()); + } + out.close(); + } + + /** + * Guts of the servlet - extracted for easy testing. + */ + static void writeResponse(Configuration conf, Writer out, String format) + throws IOException, BadFormatException { + if (FORMAT_JSON.equals(format)) { + Configuration.dumpConfiguration(conf, out); + } else if (FORMAT_XML.equals(format)) { + conf.writeXml(out); + } else { + throw new BadFormatException("Bad format: " + format); + } + } + + public static class BadFormatException extends Exception { + private static final long serialVersionUID = 1L; + + public BadFormatException(String msg) { + super(msg); + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/jmx/JMXJsonServlet.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/jmx/JMXJsonServlet.java new file mode 100644 index 00000000..4b93ab5e --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/jmx/JMXJsonServlet.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.rest.http.jmx; + +import org.apache.hadoop.hbase.rest.http.HttpServer; +import org.apache.hadoop.hbase.rest.util.JSONBean; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.management.openmbean.CompositeData; +import javax.management.openmbean.TabularData; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.io.PrintWriter; +import java.lang.management.ManagementFactory; + +/* + * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has + * been rewritten to be read only and to output in a JSON format so it is not + * really that close to the original. + */ + +/** + * Provides Read only web access to JMX. + *

+ * This servlet generally will be placed under the /jmx URL for each + * HttpServer. It provides read only + * access to JMX metrics. The optional qry parameter + * may be used to query only a subset of the JMX Beans. This query + * functionality is provided through the + * {@link MBeanServer#queryNames(ObjectName, javax.management.QueryExp)} + * method. + *

+ *

+ * For example http://.../jmx?qry=Hadoop:* will return + * all hadoop metrics exposed through JMX. + *

+ *

+ * The optional get parameter is used to query an specific + * attribute of a JMX bean. The format of the URL is + * http://.../jmx?get=MXBeanName::AttributeName + *

+ *

+ * For example + * + * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId + * will return the cluster id of the namenode mxbean. + *

+ *

+ * If the qry or the get parameter is not formatted + * correctly then a 400 BAD REQUEST http response code will be returned. + *

+ *

+ * If a resouce such as a mbean or attribute can not be found, + * a 404 SC_NOT_FOUND http response code will be returned. + *

+ *

+ * The return format is JSON and in the form + *

+ *

+ *  {
+ *    "beans" : [
+ *      {
+ *        "name":"bean-name"
+ *        ...
+ *      }
+ *    ]
+ *  }
+ *  
+ *

+ * The servlet attempts to convert the the JMXBeans into JSON. Each + * bean's attributes will be converted to a JSON object member. + * + * If the attribute is a boolean, a number, a string, or an array + * it will be converted to the JSON equivalent. + * + * If the value is a {@link CompositeData} then it will be converted + * to a JSON object with the keys as the name of the JSON member and + * the value is converted following these same rules. + * + * If the value is a {@link TabularData} then it will be converted + * to an array of the {@link CompositeData} elements that it contains. + * + * All other objects will be converted to a string and output as such. + * + * The bean's name and modelerType will be returned for all beans. + * + * Optional paramater "callback" should be used to deliver JSONP response. + *

+ * + */ +@InterfaceAudience.Private +public class JMXJsonServlet extends HttpServlet { + private static final Logger LOG = LoggerFactory.getLogger( + JMXJsonServlet.class); + + private static final long serialVersionUID = 1L; + + private static final String CALLBACK_PARAM = "callback"; + /** + * If query string includes 'description', then we will emit bean and attribute descriptions to + * output IFF they are not null and IFF the description is not the same as the attribute name: + * i.e. specify a URL like so: /jmx?description=true + */ + private static final String INCLUDE_DESCRIPTION = "description"; + + /** + * MBean server. + */ + protected transient MBeanServer mBeanServer; + + protected transient JSONBean jsonBeanWriter; + + /** + * Initialize this servlet. + */ + @Override + public void init() throws ServletException { + // Retrieve the MBean server + mBeanServer = ManagementFactory.getPlatformMBeanServer(); + this.jsonBeanWriter = new JSONBean(); + } + + /** + * Process a GET request for the specified resource. + * + * @param request + * The servlet request we are processing + * @param response + * The servlet response we are creating + */ + @Override + public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException { + try { + if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) { + return; + } + String jsonpcb = null; + PrintWriter writer = null; + JSONBean.Writer beanWriter = null; + try { + jsonpcb = checkCallbackName(request.getParameter(CALLBACK_PARAM)); + writer = response.getWriter(); + + // "callback" parameter implies JSONP outpout + if (jsonpcb != null) { + response.setContentType("application/javascript; charset=utf8"); + writer.write(jsonpcb + "("); + } else { + response.setContentType("application/json; charset=utf8"); + } + beanWriter = this.jsonBeanWriter.open(writer); + // Should we output description on each attribute and bean? + String tmpStr = request.getParameter(INCLUDE_DESCRIPTION); + boolean description = tmpStr != null && tmpStr.length() > 0; + + // query per mbean attribute + String getmethod = request.getParameter("get"); + if (getmethod != null) { + String[] splitStrings = getmethod.split("\\:\\:"); + if (splitStrings.length != 2) { + beanWriter.write("result", "ERROR"); + beanWriter.write("message", "query format is not as expected."); + beanWriter.flush(); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + return; + } + if (beanWriter.write(this.mBeanServer, new ObjectName(splitStrings[0]), + splitStrings[1], description) != 0) { + beanWriter.flush(); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + } + return; + } + + // query per mbean + String qry = request.getParameter("qry"); + if (qry == null) { + qry = "*:*"; + } + if (beanWriter.write(this.mBeanServer, new ObjectName(qry), null, description) != 0) { + beanWriter.flush(); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + } + } finally { + if (beanWriter != null) { + beanWriter.close(); + } + if (jsonpcb != null) { + writer.write(");"); + } + if (writer != null) { + writer.close(); + } + } + } catch (IOException e) { + LOG.error("Caught an exception while processing JMX request", e); + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } catch (MalformedObjectNameException e) { + LOG.error("Caught an exception while processing JMX request", e); + response.sendError(HttpServletResponse.SC_BAD_REQUEST); + } + } + + /** + * Verifies that the callback property, if provided, is purely alphanumeric. + * This prevents a malicious callback name (that is javascript code) from being + * returned by the UI to an unsuspecting user. + * + * @param callbackName The callback name, can be null. + * @return The callback name + * @throws IOException If the name is disallowed. + */ + private String checkCallbackName(String callbackName) throws IOException { + if (null == callbackName) { + return null; + } + if (callbackName.matches("[A-Za-z0-9_]+")) { + return callbackName; + } + throw new IOException("'callback' must be alphanumeric"); + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/Log4jUtils.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/Log4jUtils.java new file mode 100644 index 00000000..368167e1 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/Log4jUtils.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http.log; + +import org.apache.yetus.audience.InterfaceAudience; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Set; + +/** + * A bridge class for operating on log4j, such as changing log level, etc. + *

+ * Will call the methods in {@link InternalLog4jUtils} to actually operate on the log4j stuff. + */ +@InterfaceAudience.Private +public final class Log4jUtils { + + private static final String INTERNAL_UTILS_CLASS_NAME = + "org.apache.hadoop.hbase.logging.InternalLog4jUtils"; + + private Log4jUtils() { + } + + // load class when calling to avoid introducing class not found exception on log4j when loading + // this class even without calling any of the methods below. + private static Method getMethod(String methodName, Class... args) { + try { + Class clazz = Class.forName(INTERNAL_UTILS_CLASS_NAME); + return clazz.getDeclaredMethod(methodName, args); + } catch (ClassNotFoundException | NoSuchMethodException e) { + throw new AssertionError("should not happen", e); + } + } + + private static void throwUnchecked(Throwable throwable) { + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } + if (throwable instanceof Error) { + throw (Error) throwable; + } + } + + public static void setLogLevel(String loggerName, String levelName) { + Method method = getMethod("setLogLevel", String.class, String.class); + try { + method.invoke(null, loggerName, levelName); + } catch (IllegalAccessException e) { + throw new AssertionError("should not happen", e); + } catch (InvocationTargetException e) { + throwUnchecked(e.getCause()); + throw new AssertionError("should not happen", e.getCause()); + } + } + + public static String getEffectiveLevel(String loggerName) { + Method method = getMethod("getEffectiveLevel", String.class); + try { + return (String) method.invoke(null, loggerName); + } catch (IllegalAccessException e) { + throw new AssertionError("should not happen", e); + } catch (InvocationTargetException e) { + throwUnchecked(e.getCause()); + throw new AssertionError("should not happen", e.getCause()); + } + } + + @SuppressWarnings("unchecked") + public static Set getActiveLogFiles() throws IOException { + Method method = getMethod("getActiveLogFiles"); + try { + return (Set) method.invoke(null); + } catch (IllegalAccessException e) { + throw new AssertionError("should not happen", e); + } catch (InvocationTargetException e) { + Throwable cause = e.getCause(); + throwUnchecked(cause); + if (cause instanceof IOException) { + throw (IOException) cause; + } + throw new AssertionError("should not happen", cause); + } + } + + /** + * Disables Zk- and HBase client logging + */ + public static void disableZkAndClientLoggers() { + // disable zookeeper log to avoid it mess up command output + setLogLevel("org.apache.zookeeper", "OFF"); + // disable hbase zookeeper tool log to avoid it mess up command output + setLogLevel("org.apache.hadoop.hbase.zookeeper", "OFF"); + // disable hbase client log to avoid it mess up command output + setLogLevel("org.apache.hadoop.hbase.client", "OFF"); + } + + /** + * Switches the logger for the given class to DEBUG level. + * @param clazz The class for which to switch to debug logging. + */ + public static void enableDebug(Class clazz) { + setLogLevel(clazz.getName(), "DEBUG"); + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/LogLevel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/LogLevel.java new file mode 100644 index 00000000..26a0c72d --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/http/log/LogLevel.java @@ -0,0 +1,401 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.http.log; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.rest.http.HttpServer; +import org.apache.hadoop.security.authentication.client.AuthenticatedURL; +import org.apache.hadoop.security.authentication.client.KerberosAuthenticator; +import org.apache.hadoop.security.ssl.SSLFactory; +import org.apache.hadoop.util.ServletUtil; +import org.apache.hadoop.util.Tool; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Charsets; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLSocketFactory; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.BufferedReader; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.PrintWriter; +import java.net.URL; +import java.net.URLConnection; +import java.util.Objects; +import java.util.regex.Pattern; + +/** + * Change log level in runtime. + */ +@InterfaceAudience.Private +public final class LogLevel { + private static final String USAGES = "\nUsage: General options are:\n" + + "\t[-getlevel [-protocol (http|https)]\n" + + "\t[-setlevel [-protocol (http|https)]"; + + public static final String PROTOCOL_HTTP = "http"; + public static final String PROTOCOL_HTTPS = "https"; + + /** + * A command line implementation + */ + public static void main(String[] args) throws Exception { + CLI cli = new CLI(new Configuration()); + System.exit(cli.run(args)); + } + + /** + * Valid command line options. + */ + private enum Operations { + GETLEVEL, + SETLEVEL, + UNKNOWN + } + + private static void printUsage() { + System.err.println(USAGES); + System.exit(-1); + } + + public static boolean isValidProtocol(String protocol) { + return ((protocol.equals(PROTOCOL_HTTP) || + protocol.equals(PROTOCOL_HTTPS))); + } + + @VisibleForTesting + static class CLI extends Configured implements Tool { + private Operations operation = Operations.UNKNOWN; + private String protocol; + private String hostName; + private String className; + private String level; + + CLI(Configuration conf) { + setConf(conf); + } + + @Override + public int run(String[] args) throws Exception { + try { + parseArguments(args); + sendLogLevelRequest(); + } catch (HadoopIllegalArgumentException e) { + printUsage(); + } + return 0; + } + + /** + * Send HTTP request to the daemon. + * @throws HadoopIllegalArgumentException if arguments are invalid. + * @throws Exception if unable to connect + */ + private void sendLogLevelRequest() + throws HadoopIllegalArgumentException, Exception { + switch (operation) { + case GETLEVEL: + doGetLevel(); + break; + case SETLEVEL: + doSetLevel(); + break; + default: + throw new HadoopIllegalArgumentException( + "Expect either -getlevel or -setlevel"); + } + } + + public void parseArguments(String[] args) throws HadoopIllegalArgumentException { + if (args.length == 0) { + throw new HadoopIllegalArgumentException("No arguments specified"); + } + int nextArgIndex = 0; + while (nextArgIndex < args.length) { + switch (args[nextArgIndex]) { + case "-getlevel": + nextArgIndex = parseGetLevelArgs(args, nextArgIndex); + break; + case "-setlevel": + nextArgIndex = parseSetLevelArgs(args, nextArgIndex); + break; + case "-protocol": + nextArgIndex = parseProtocolArgs(args, nextArgIndex); + break; + default: + throw new HadoopIllegalArgumentException( + "Unexpected argument " + args[nextArgIndex]); + } + } + + // if operation is never specified in the arguments + if (operation == Operations.UNKNOWN) { + throw new HadoopIllegalArgumentException( + "Must specify either -getlevel or -setlevel"); + } + + // if protocol is unspecified, set it as http. + if (protocol == null) { + protocol = PROTOCOL_HTTP; + } + } + + private int parseGetLevelArgs(String[] args, int index) throws HadoopIllegalArgumentException { + // fail if multiple operations are specified in the arguments + if (operation != Operations.UNKNOWN) { + throw new HadoopIllegalArgumentException("Redundant -getlevel command"); + } + // check number of arguments is sufficient + if (index + 2 >= args.length) { + throw new HadoopIllegalArgumentException("-getlevel needs two parameters"); + } + operation = Operations.GETLEVEL; + hostName = args[index + 1]; + className = args[index + 2]; + return index + 3; + } + + private int parseSetLevelArgs(String[] args, int index) throws HadoopIllegalArgumentException { + // fail if multiple operations are specified in the arguments + if (operation != Operations.UNKNOWN) { + throw new HadoopIllegalArgumentException("Redundant -setlevel command"); + } + // check number of arguments is sufficient + if (index + 3 >= args.length) { + throw new HadoopIllegalArgumentException("-setlevel needs three parameters"); + } + operation = Operations.SETLEVEL; + hostName = args[index + 1]; + className = args[index + 2]; + level = args[index + 3]; + return index + 4; + } + + private int parseProtocolArgs(String[] args, int index) throws HadoopIllegalArgumentException { + // make sure only -protocol is specified + if (protocol != null) { + throw new HadoopIllegalArgumentException( + "Redundant -protocol command"); + } + // check number of arguments is sufficient + if (index + 1 >= args.length) { + throw new HadoopIllegalArgumentException( + "-protocol needs one parameter"); + } + // check protocol is valid + protocol = args[index + 1]; + if (!isValidProtocol(protocol)) { + throw new HadoopIllegalArgumentException( + "Invalid protocol: " + protocol); + } + return index + 2; + } + + /** + * Send HTTP request to get log level. + * + * @throws HadoopIllegalArgumentException if arguments are invalid. + * @throws Exception if unable to connect + */ + private void doGetLevel() throws Exception { + process(protocol + "://" + hostName + "/logLevel?log=" + className); + } + + /** + * Send HTTP request to set log level. + * + * @throws HadoopIllegalArgumentException if arguments are invalid. + * @throws Exception if unable to connect + */ + private void doSetLevel() throws Exception { + process(protocol + "://" + hostName + "/logLevel?log=" + className + + "&level=" + level); + } + + /** + * Connect to the URL. Supports HTTP and supports SPNEGO + * authentication. It falls back to simple authentication if it fails to + * initiate SPNEGO. + * + * @param url the URL address of the daemon servlet + * @return a connected connection + * @throws Exception if it can not establish a connection. + */ + private URLConnection connect(URL url) throws Exception { + AuthenticatedURL.Token token = new AuthenticatedURL.Token(); + AuthenticatedURL aUrl; + SSLFactory clientSslFactory; + URLConnection connection; + // If https is chosen, configures SSL client. + if (PROTOCOL_HTTPS.equals(url.getProtocol())) { + clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, this.getConf()); + clientSslFactory.init(); + SSLSocketFactory sslSocketF = clientSslFactory.createSSLSocketFactory(); + + aUrl = new AuthenticatedURL(new KerberosAuthenticator(), clientSslFactory); + connection = aUrl.openConnection(url, token); + HttpsURLConnection httpsConn = (HttpsURLConnection) connection; + httpsConn.setSSLSocketFactory(sslSocketF); + } else { + aUrl = new AuthenticatedURL(new KerberosAuthenticator()); + connection = aUrl.openConnection(url, token); + } + connection.connect(); + return connection; + } + + /** + * Configures the client to send HTTP request to the URL. + * Supports SPENGO for authentication. + * @param urlString URL and query string to the daemon's web UI + * @throws Exception if unable to connect + */ + private void process(String urlString) throws Exception { + URL url = new URL(urlString); + System.out.println("Connecting to " + url); + + URLConnection connection = connect(url); + + // read from the servlet + + try (InputStreamReader streamReader = + new InputStreamReader(connection.getInputStream(), Charsets.UTF_8); + BufferedReader bufferedReader = new BufferedReader(streamReader)) { + bufferedReader.lines().filter(Objects::nonNull).filter(line -> line.startsWith(MARKER)) + .forEach(line -> System.out.println(TAG.matcher(line).replaceAll(""))); + } catch (IOException ioe) { + System.err.println("" + ioe); + } + } + } + + private static final String MARKER = ""; + private static final Pattern TAG = Pattern.compile("<[^>]*>"); + + /** + * A servlet implementation + */ + @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) + @InterfaceStability.Unstable + public static class Servlet extends HttpServlet { + private static final long serialVersionUID = 1L; + + @Override + public void doGet(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + // Do the authorization + if (!HttpServer.hasAdministratorAccess(getServletContext(), request, + response)) { + return; + } + // Disallow modification of the LogLevel if explicitly set to readonly + Configuration conf = (Configuration) getServletContext().getAttribute( + HttpServer.CONF_CONTEXT_ATTRIBUTE); + if (conf.getBoolean("hbase.master.ui.readonly", false)) { + response.sendError(HttpServletResponse.SC_FORBIDDEN, "Modification of HBase via" + + " the UI is disallowed in configuration."); + return; + } + response.setContentType("text/html"); + PrintWriter out; + try { + String headerPath = "header.jsp?pageTitle=Log Level"; + request.getRequestDispatcher(headerPath).include(request, response); + out = response.getWriter(); + } catch (FileNotFoundException e) { + // in case file is not found fall back to old design + out = ServletUtil.initHTML(response, "Log Level"); + } + out.println(FORMS); + + String logName = ServletUtil.getParameter(request, "log"); + String level = ServletUtil.getParameter(request, "level"); + + if (logName != null) { + out.println("

Results:

"); + out.println(MARKER + + "Submitted Log Name: " + logName + "
"); + + Logger log = LoggerFactory.getLogger(logName); + out.println(MARKER + + "Log Class: " + log.getClass().getName() +"
"); + if (level != null) { + out.println(MARKER + "Submitted Level: " + level + "
"); + } + process(log, level, out); + } + + try { + String footerPath = "footer.jsp"; + out.println(""); + request.getRequestDispatcher(footerPath).include(request, response); + } catch (FileNotFoundException e) { + out.println(ServletUtil.HTML_TAIL); + } + out.close(); + } + + static final String FORMS = "
\n" + + "
\n" + "\n" + "
\n" + "Actions:" + "

" + + "

\n" + "\n" + "\n" + + "\n" + "\n" + "\n" + "\n" + "\n" + + "\n" + "\n" + "\n" + "\n" + + "\n" + "\n" + "\n" + + "\n" + "
\n" + + "\n" + "\n" + + "\n" + "" + + "Get the current log level for the specified log name." + "
\n" + + "\n" + "\n" + + "\n" + + "\n" + "" + + "Set the specified log level for the specified log name." + "
\n" + "
\n" + "

\n" + "
\n"; + + private static void process(Logger logger, String levelName, PrintWriter out) { + if (levelName != null) { + try { + Log4jUtils.setLogLevel(logger.getName(), levelName); + out.println(MARKER + "
" + "Setting Level to " + + levelName + " ...
" + "
"); + } catch (IllegalArgumentException e) { + out.println(MARKER + "
" + "Bad level : " + levelName + + "
" + "
"); + } + } + out.println(MARKER + "Effective level: " + Log4jUtils.getEffectiveLevel(logger.getName()) + + "
"); + } + } + + private LogLevel() {} +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ConnectionCache.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ConnectionCache.java new file mode 100644 index 00000000..cbd58746 --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ConnectionCache.java @@ -0,0 +1,245 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.util; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; + +/** + * A utility to store user specific HConnections in memory. + * There is a chore to clean up connections idle for too long. + * This class is used by REST server and Thrift server to + * support authentication and impersonation. + */ +@InterfaceAudience.Private +public class ConnectionCache { + private static final Logger LOG = LoggerFactory.getLogger(ConnectionCache.class); + + private final Map connections = new ConcurrentHashMap<>(); + private final KeyLocker locker = new KeyLocker<>(); + private final String realUserName; + private final UserGroupInformation realUser; + private final UserProvider userProvider; + private final Configuration conf; + private final ChoreService choreService; + + private final ThreadLocal effectiveUserNames = + new ThreadLocal() { + @Override + protected String initialValue() { + return realUserName; + } + }; + + public ConnectionCache(final Configuration conf, + final UserProvider userProvider, + final int cleanInterval, final int maxIdleTime) throws IOException { + Stoppable stoppable = new Stoppable() { + private volatile boolean isStopped = false; + @Override public void stop(String why) { isStopped = true;} + @Override public boolean isStopped() {return isStopped;} + }; + this.choreService = new ChoreService("ConnectionCache"); + ScheduledChore cleaner = new ScheduledChore("ConnectionCleaner", stoppable, cleanInterval) { + @Override + protected void chore() { + for (Map.Entry entry: connections.entrySet()) { + ConnectionInfo connInfo = entry.getValue(); + if (connInfo.timedOut(maxIdleTime)) { + if (connInfo.admin != null) { + try { + connInfo.admin.close(); + } catch (Throwable t) { + LOG.info("Got exception in closing idle admin", t); + } + } + try { + connInfo.connection.close(); + } catch (Throwable t) { + LOG.info("Got exception in closing idle connection", t); + } + } + } + } + }; + // Start the daemon cleaner chore + choreService.scheduleChore(cleaner); + this.realUser = userProvider.getCurrent().getUGI(); + this.realUserName = realUser.getShortUserName(); + this.userProvider = userProvider; + this.conf = conf; + } + + /** + * Set the current thread local effective user + */ + public void setEffectiveUser(String user) { + effectiveUserNames.set(user); + } + + /** + * Get the current thread local effective user + */ + public String getEffectiveUser() { + return effectiveUserNames.get(); + } + + /** + * Called when cache is no longer needed so that it can perform cleanup operations + */ + public void shutdown() { + if (choreService != null) choreService.shutdown(); + } + + /** + * Caller doesn't close the admin afterwards. + * We need to manage it and close it properly. + */ + public Admin getAdmin() throws IOException { + ConnectionInfo connInfo = getCurrentConnection(); + if (connInfo.admin == null) { + Lock lock = locker.acquireLock(getEffectiveUser()); + try { + if (connInfo.admin == null) { + connInfo.admin = connInfo.connection.getAdmin(); + } + } finally { + lock.unlock(); + } + } + return connInfo.admin; + } + + /** + * Caller closes the table afterwards. + */ + public Table getTable(String tableName) throws IOException { + ConnectionInfo connInfo = getCurrentConnection(); + return connInfo.connection.getTable(TableName.valueOf(tableName)); + } + + /** + * Retrieve a regionLocator for the table. The user should close the RegionLocator. + */ + public RegionLocator getRegionLocator(byte[] tableName) throws IOException { + return getCurrentConnection().connection.getRegionLocator(TableName.valueOf(tableName)); + } + + /** + * Get the cached connection for the current user. + * If none or timed out, create a new one. + */ + ConnectionInfo getCurrentConnection() throws IOException { + String userName = getEffectiveUser(); + ConnectionInfo connInfo = connections.get(userName); + if (connInfo == null || !connInfo.updateAccessTime()) { + Lock lock = locker.acquireLock(userName); + try { + connInfo = connections.get(userName); + if (connInfo == null) { + UserGroupInformation ugi = realUser; + if (!userName.equals(realUserName)) { + ugi = UserGroupInformation.createProxyUser(userName, realUser); + } + User user = userProvider.create(ugi); + Connection conn = ConnectionFactory.createConnection(conf, user); + connInfo = new ConnectionInfo(conn, userName); + connections.put(userName, connInfo); + } + } finally { + lock.unlock(); + } + } + return connInfo; + } + + /** + * Updates the access time for the current connection. Used to keep Connections alive for + * long-lived scanners. + * @return whether we successfully updated the last access time + */ + public boolean updateConnectionAccessTime() { + String userName = getEffectiveUser(); + ConnectionInfo connInfo = connections.get(userName); + if (connInfo != null) { + return connInfo.updateAccessTime(); + } + return false; + } + + class ConnectionInfo { + final Connection connection; + final String userName; + + volatile Admin admin; + private long lastAccessTime; + private boolean closed; + + ConnectionInfo(Connection conn, String user) { + lastAccessTime = EnvironmentEdgeManager.currentTime(); + connection = conn; + closed = false; + userName = user; + } + + synchronized boolean updateAccessTime() { + if (closed) { + return false; + } + if (connection.isAborted() || connection.isClosed()) { + LOG.info("Unexpected: cached Connection is aborted/closed, removed from cache"); + connections.remove(userName); + return false; + } + lastAccessTime = EnvironmentEdgeManager.currentTime(); + return true; + } + + synchronized boolean timedOut(int maxIdleTime) { + long timeoutTime = lastAccessTime + maxIdleTime; + if (EnvironmentEdgeManager.currentTime() > timeoutTime) { + connections.remove(userName); + closed = true; + return true; + } + return false; + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JSONBean.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JSONBean.java new file mode 100644 index 00000000..d28641fd --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JSONBean.java @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.util; + +import org.apache.hadoop.hbase.util.GsonUtil; +import org.apache.hbase.thirdparty.com.google.gson.Gson; +import org.apache.hbase.thirdparty.com.google.gson.stream.JsonWriter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.AttributeNotFoundException; +import javax.management.InstanceNotFoundException; +import javax.management.IntrospectionException; +import javax.management.MBeanAttributeInfo; +import javax.management.MBeanException; +import javax.management.MBeanInfo; +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.management.ReflectionException; +import javax.management.RuntimeErrorException; +import javax.management.RuntimeMBeanException; +import javax.management.openmbean.CompositeData; +import javax.management.openmbean.CompositeType; +import javax.management.openmbean.TabularData; +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.lang.management.ManagementFactory; +import java.lang.reflect.Array; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.Set; + +/** + * Utility for doing JSON and MBeans. + */ +@InterfaceAudience.Private +public class JSONBean { + private static final Logger LOG = LoggerFactory.getLogger(JSONBean.class); + private static final Gson GSON = GsonUtil.createGson().create(); + + /** + * Use dumping out mbeans as JSON. + */ + public interface Writer extends Closeable { + + void write(String key, String value) throws IOException; + + int write(MBeanServer mBeanServer, ObjectName qry, String attribute, boolean description) + throws IOException; + + void flush() throws IOException; + } + + /** + * Notice that, closing the return {@link Writer} will not close the {@code writer} passed in, you + * still need to close the {@code writer} by yourself. + *

+ * This is because that, we can only finish the json after you call {@link Writer#close()}. So if + * we just close the {@code writer}, you can write nothing after finished the json. + */ + public Writer open(final PrintWriter writer) throws IOException { + JsonWriter jsonWriter = GSON.newJsonWriter(new java.io.Writer() { + + @Override + public void write(char[] cbuf, int off, int len) throws IOException { + writer.write(cbuf, off, len); + } + + @Override + public void flush() throws IOException { + writer.flush(); + } + + @Override + public void close() throws IOException { + // do nothing + } + }); + jsonWriter.setIndent(" "); + jsonWriter.beginObject(); + return new Writer() { + @Override + public void flush() throws IOException { + jsonWriter.flush(); + } + + @Override + public void close() throws IOException { + jsonWriter.endObject(); + jsonWriter.close(); + } + + @Override + public void write(String key, String value) throws IOException { + jsonWriter.name(key).value(value); + } + + @Override + public int write(MBeanServer mBeanServer, ObjectName qry, String attribute, + boolean description) throws IOException { + return JSONBean.write(jsonWriter, mBeanServer, qry, attribute, description); + } + }; + } + + /** + * @return Return non-zero if failed to find bean. 0 + */ + private static int write(JsonWriter writer, MBeanServer mBeanServer, ObjectName qry, + String attribute, boolean description) throws IOException { + LOG.trace("Listing beans for " + qry); + Set names = null; + names = mBeanServer.queryNames(qry, null); + writer.name("beans").beginArray(); + Iterator it = names.iterator(); + while (it.hasNext()) { + ObjectName oname = it.next(); + MBeanInfo minfo; + String code = ""; + String descriptionStr = null; + Object attributeinfo = null; + try { + minfo = mBeanServer.getMBeanInfo(oname); + code = minfo.getClassName(); + if (description) { + descriptionStr = minfo.getDescription(); + } + String prs = ""; + try { + if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) { + prs = "modelerType"; + code = (String) mBeanServer.getAttribute(oname, prs); + } + if (attribute != null) { + prs = attribute; + attributeinfo = mBeanServer.getAttribute(oname, prs); + } + } catch (RuntimeMBeanException e) { + // UnsupportedOperationExceptions happen in the normal course of business, + // so no need to log them as errors all the time. + if (e.getCause() instanceof UnsupportedOperationException) { + if (LOG.isTraceEnabled()) { + LOG.trace("Getting attribute " + prs + " of " + oname + " threw " + e); + } + } else { + LOG.error("Getting attribute " + prs + " of " + oname + " threw an exception", e); + } + return 0; + } catch (AttributeNotFoundException e) { + // If the modelerType attribute was not found, the class name is used + // instead. + LOG.error("getting attribute " + prs + " of " + oname + " threw an exception", e); + } catch (MBeanException e) { + // The code inside the attribute getter threw an exception so log it, + // and fall back on the class name + LOG.error("getting attribute " + prs + " of " + oname + " threw an exception", e); + } catch (RuntimeException e) { + // For some reason even with an MBeanException available to them + // Runtime exceptionscan still find their way through, so treat them + // the same as MBeanException + LOG.error("getting attribute " + prs + " of " + oname + " threw an exception", e); + } catch (ReflectionException e) { + // This happens when the code inside the JMX bean (setter?? from the + // java docs) threw an exception, so log it and fall back on the + // class name + LOG.error("getting attribute " + prs + " of " + oname + " threw an exception", e); + } + } catch (InstanceNotFoundException e) { + // Ignored for some reason the bean was not found so don't output it + continue; + } catch (IntrospectionException e) { + // This is an internal error, something odd happened with reflection so + // log it and don't output the bean. + LOG.error("Problem while trying to process JMX query: " + qry + " with MBean " + oname, e); + continue; + } catch (ReflectionException e) { + // This happens when the code inside the JMX bean threw an exception, so + // log it and don't output the bean. + LOG.error("Problem while trying to process JMX query: " + qry + " with MBean " + oname, e); + continue; + } + writer.beginObject(); + writer.name("name").value(oname.toString()); + if (description && descriptionStr != null && descriptionStr.length() > 0) { + writer.name("description").value(descriptionStr); + } + writer.name("modelerType").value(code); + if (attribute != null && attributeinfo == null) { + writer.name("result").value("ERROR"); + writer.name("message").value("No attribute with name " + attribute + " was found."); + writer.endObject(); + writer.endArray(); + writer.close(); + return -1; + } + + if (attribute != null) { + writeAttribute(writer, attribute, descriptionStr, attributeinfo); + } else { + MBeanAttributeInfo[] attrs = minfo.getAttributes(); + for (int i = 0; i < attrs.length; i++) { + writeAttribute(writer, mBeanServer, oname, description, attrs[i]); + } + } + writer.endObject(); + } + writer.endArray(); + return 0; + } + + private static void writeAttribute(JsonWriter writer, MBeanServer mBeanServer, ObjectName oname, + boolean description, MBeanAttributeInfo attr) throws IOException { + if (!attr.isReadable()) { + return; + } + String attName = attr.getName(); + if ("modelerType".equals(attName)) { + return; + } + if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0 || attName.indexOf(" ") >= 0) { + return; + } + String descriptionStr = description ? attr.getDescription() : null; + Object value = null; + try { + value = mBeanServer.getAttribute(oname, attName); + } catch (RuntimeMBeanException e) { + // UnsupportedOperationExceptions happen in the normal course of business, + // so no need to log them as errors all the time. + if (e.getCause() instanceof UnsupportedOperationException) { + if (LOG.isTraceEnabled()) { + LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + e); + } + } else { + LOG.error("getting attribute " + attName + " of " + oname + " threw an exception", e); + } + return; + } catch (RuntimeErrorException e) { + // RuntimeErrorException happens when an unexpected failure occurs in getAttribute + // for example https://issues.apache.org/jira/browse/DAEMON-120 + LOG.debug("getting attribute " + attName + " of " + oname + " threw an exception", e); + return; + } catch (AttributeNotFoundException e) { + // Ignored the attribute was not found, which should never happen because the bean + // just told us that it has this attribute, but if this happens just don't output + // the attribute. + return; + } catch (MBeanException e) { + // The code inside the attribute getter threw an exception so log it, and + // skip outputting the attribute + LOG.error("getting attribute " + attName + " of " + oname + " threw an exception", e); + return; + } catch (RuntimeException e) { + // For some reason even with an MBeanException available to them Runtime exceptions + // can still find their way through, so treat them the same as MBeanException + LOG.error("getting attribute " + attName + " of " + oname + " threw an exception", e); + return; + } catch (ReflectionException e) { + // This happens when the code inside the JMX bean (setter?? from the java docs) + // threw an exception, so log it and skip outputting the attribute + LOG.error("getting attribute " + attName + " of " + oname + " threw an exception", e); + return; + } catch (InstanceNotFoundException e) { + // Ignored the mbean itself was not found, which should never happen because we + // just accessed it (perhaps something unregistered in-between) but if this + // happens just don't output the attribute. + return; + } + + writeAttribute(writer, attName, descriptionStr, value); + } + + private static void writeAttribute(JsonWriter writer, String attName, String descriptionStr, + Object value) throws IOException { + if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) { + writer.name(attName); + writer.beginObject(); + writer.name("description").value(descriptionStr); + writer.name("value"); + writeObject(writer, value); + writer.endObject(); + } else { + writer.name(attName); + writeObject(writer, value); + } + } + + private static void writeObject(JsonWriter writer, Object value) throws IOException { + if (value == null) { + writer.nullValue(); + } else { + Class c = value.getClass(); + if (c.isArray()) { + writer.beginArray(); + int len = Array.getLength(value); + for (int j = 0; j < len; j++) { + Object item = Array.get(value, j); + writeObject(writer, item); + } + writer.endArray(); + } else if (value instanceof Number) { + Number n = (Number) value; + if (Double.isFinite(n.doubleValue())) { + writer.value(n); + } else { + writer.value(n.toString()); + } + } else if (value instanceof Boolean) { + Boolean b = (Boolean) value; + writer.value(b); + } else if (value instanceof CompositeData) { + CompositeData cds = (CompositeData) value; + CompositeType comp = cds.getCompositeType(); + Set keys = comp.keySet(); + writer.beginObject(); + for (String key : keys) { + writeAttribute(writer, key, null, cds.get(key)); + } + writer.endObject(); + } else if (value instanceof TabularData) { + TabularData tds = (TabularData) value; + writer.beginArray(); + for (Object entry : tds.values()) { + writeObject(writer, entry); + } + writer.endArray(); + } else { + writer.value(value.toString()); + } + } + } + + /** + * Dump out all registered mbeans as json on System.out. + */ + public static void dumpAllBeans() throws IOException, MalformedObjectNameException { + try (PrintWriter writer = + new PrintWriter(new OutputStreamWriter(System.out, StandardCharsets.UTF_8))) { + JSONBean dumper = new JSONBean(); + try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) { + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + jsonBeanWriter.write(mbeanServer, new ObjectName("*:*"), null, false); + } + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JvmPauseMonitor.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JvmPauseMonitor.java new file mode 100644 index 00000000..4c95d43c --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/JvmPauseMonitor.java @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.util; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.metrics.JvmPauseMonitorSource; +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.management.GarbageCollectorMXBean; +import java.lang.management.ManagementFactory; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * Class which sets up a simple thread which runs in a loop sleeping + * for a short interval of time. If the sleep takes significantly longer + * than its target time, it implies that the JVM or host machine has + * paused processing, which may cause other problems. If such a pause is + * detected, the thread logs a message. + * The original JvmPauseMonitor is: + * ${hadoop-common-project}/hadoop-common/src/main/java/org/apache/hadoop/util/ + * JvmPauseMonitor.java + * r1503806 | cmccabe | 2013-07-17 01:48:24 +0800 (Wed, 17 Jul 2013) | 1 line + * HADOOP-9618. thread which detects GC pauses(Todd Lipcon) + */ +@InterfaceAudience.Private +public class JvmPauseMonitor { + private static final Logger LOG = LoggerFactory.getLogger(JvmPauseMonitor.class); + + /** The target sleep time */ + private static final long SLEEP_INTERVAL_MS = 500; + + /** log WARN if we detect a pause longer than this threshold */ + private final long warnThresholdMs; + public static final String WARN_THRESHOLD_KEY = + "jvm.pause.warn-threshold.ms"; + private static final long WARN_THRESHOLD_DEFAULT = 10000; + + /** log INFO if we detect a pause longer than this threshold */ + private final long infoThresholdMs; + public static final String INFO_THRESHOLD_KEY = + "jvm.pause.info-threshold.ms"; + private static final long INFO_THRESHOLD_DEFAULT = 1000; + + private Thread monitorThread; + private volatile boolean shouldRun = true; + private JvmPauseMonitorSource metricsSource; + + public JvmPauseMonitor(Configuration conf) { + this(conf, null); + } + + public JvmPauseMonitor(Configuration conf, JvmPauseMonitorSource metricsSource) { + this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT); + this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT); + this.metricsSource = metricsSource; + } + + public void start() { + Preconditions.checkState(monitorThread == null, "Already started"); + monitorThread = new Thread(new Monitor(), "JvmPauseMonitor"); + monitorThread.setDaemon(true); + monitorThread.start(); + } + + public void stop() { + shouldRun = false; + monitorThread.interrupt(); + try { + monitorThread.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private String formatMessage(long extraSleepTime, List gcDiffs) { + String ret = "Detected pause in JVM or host machine (eg GC): " + "pause of approximately " + + extraSleepTime + "ms\n"; + if (gcDiffs.isEmpty()) { + ret += "No GCs detected"; + } else { + ret += Joiner.on("\n").join(gcDiffs); + } + return ret; + } + + private Map getGcTimes() { + Map map = Maps.newHashMap(); + List gcBeans = ManagementFactory.getGarbageCollectorMXBeans(); + for (GarbageCollectorMXBean gcBean : gcBeans) { + map.put(gcBean.getName(), new GcTimes(gcBean)); + } + return map; + } + + private static class GcTimes { + private GcTimes(GarbageCollectorMXBean gcBean) { + gcCount = gcBean.getCollectionCount(); + gcTimeMillis = gcBean.getCollectionTime(); + } + + private GcTimes(long count, long time) { + this.gcCount = count; + this.gcTimeMillis = time; + } + + private GcTimes subtract(GcTimes other) { + return new GcTimes(this.gcCount - other.gcCount, this.gcTimeMillis - other.gcTimeMillis); + } + + @Override + public String toString() { + return "count=" + gcCount + " time=" + gcTimeMillis + "ms"; + } + + private long gcCount; + private long gcTimeMillis; + } + + private class Monitor implements Runnable { + @Override + public void run() { + Stopwatch sw = Stopwatch.createUnstarted(); + Map gcTimesBeforeSleep = getGcTimes(); + while (shouldRun) { + sw.reset().start(); + try { + Thread.sleep(SLEEP_INTERVAL_MS); + } catch (InterruptedException ie) { + return; + } + + long extraSleepTime = sw.elapsed(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS; + Map gcTimesAfterSleep = getGcTimes(); + + if (extraSleepTime > infoThresholdMs) { + Set gcBeanNames = Sets.intersection(gcTimesAfterSleep.keySet(), + gcTimesBeforeSleep.keySet()); + List gcDiffs = Lists.newArrayList(); + for (String name : gcBeanNames) { + GcTimes diff = gcTimesAfterSleep.get(name).subtract(gcTimesBeforeSleep.get(name)); + if (diff.gcCount != 0) { + gcDiffs.add("GC pool '" + name + "' had collection(s): " + diff.toString()); + } + } + + updateMetrics(extraSleepTime, !gcDiffs.isEmpty()); + + if (extraSleepTime > warnThresholdMs) { + LOG.warn(formatMessage(extraSleepTime, gcDiffs)); + } else { + LOG.info(formatMessage(extraSleepTime, gcDiffs)); + } + } + gcTimesBeforeSleep = gcTimesAfterSleep; + } + } + } + + public void updateMetrics(long sleepTime, boolean gcDetected) { + if (metricsSource != null) { + if (sleepTime > warnThresholdMs) { + metricsSource.incWarnThresholdExceeded(1); + } else { + metricsSource.incInfoThresholdExceeded(1); + } + if (gcDetected) { + metricsSource.updatePauseTimeWithGc(sleepTime); + } else { + metricsSource.updatePauseTimeWithoutGc(sleepTime); + } + } + } + + public JvmPauseMonitorSource getMetricsSource() { + return metricsSource; + } + + public void setMetricsSource(JvmPauseMonitorSource metricsSource) { + this.metricsSource = metricsSource; + } + + /** + * Simple 'main' to facilitate manual testing of the pause monitor. + * + * This main function just leaks memory into a list. Running this class + * with a 1GB heap will very quickly go into "GC hell" and result in + * log messages about the GC pauses. + */ + public static void main(String []args) throws Exception { + new JvmPauseMonitor(new Configuration()).start(); + List list = Lists.newArrayList(); + int i = 0; + while (true) { + list.add(String.valueOf(i++)); + } + } +} diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ProcessUtils.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ProcessUtils.java new file mode 100644 index 00000000..f166672a --- /dev/null +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/util/ProcessUtils.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.rest.util; + +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.util.List; + +/** + * Process related utilities. + */ +@InterfaceAudience.Private +public final class ProcessUtils { + private static Logger LOG = LoggerFactory.getLogger(ProcessUtils.class); + + private ProcessUtils() { } + + public static Integer getPid() { + // JVM_PID is exported by bin/hbase run script + String pidStr = System.getenv("JVM_PID"); + + // in case if it is not set correctly used fallback from mxbean which is implementation specific + if (pidStr == null || pidStr.trim().isEmpty()) { + String name = ManagementFactory.getRuntimeMXBean().getName(); + if (name != null) { + int idx = name.indexOf("@"); + if (idx != -1) { + pidStr = name.substring(0, name.indexOf("@")); + } + } + } + try { + if (pidStr != null) { + return Integer.valueOf(pidStr); + } + } catch (NumberFormatException nfe) { + // ignore + } + return null; + } + + public static Process runCmdAsync(List cmd) { + try { + LOG.info("Running command async: " + cmd); + return new ProcessBuilder(cmd).inheritIO().start(); + } catch (IOException ex) { + throw new IllegalStateException(ex); + } + } +} diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java index be7bb746..4b6d23b1 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java @@ -31,7 +31,6 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Row; @@ -54,10 +52,8 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.rest.Constants; @@ -739,11 +735,6 @@ public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) { return new CheckAndMutateBuilderImpl(row, family); } - @Override - public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) { - throw new NotImplementedException("Implement later"); - } - @Override public Result increment(Increment increment) throws IOException { throw new IOException("Increment not supported"); @@ -782,38 +773,11 @@ public CoprocessorRpcChannel coprocessorService(byte[] row) { throw new UnsupportedOperationException("coprocessorService not implemented"); } - @Override - public Map coprocessorService(Class service, byte[] startKey, - byte[] endKey, Batch.Call callable) throws ServiceException, Throwable { - throw new UnsupportedOperationException("coprocessorService not implemented"); - } - - @Override - public void coprocessorService(Class service, byte[] startKey, - byte[] endKey, Batch.Call callable, Callback callback) - throws ServiceException, Throwable { - throw new UnsupportedOperationException("coprocessorService not implemented"); - } - @Override public void mutateRow(RowMutations rm) throws IOException { throw new IOException("atomicMutation not supported"); } - @Override - public Map batchCoprocessorService( - Descriptors.MethodDescriptor method, Message request, byte[] startKey, byte[] endKey, - R responsePrototype) throws ServiceException, Throwable { - throw new UnsupportedOperationException("batchCoprocessorService not implemented"); - } - - @Override - public void batchCoprocessorService(Descriptors.MethodDescriptor method, - Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback callback) - throws ServiceException, Throwable { - throw new UnsupportedOperationException("batchCoprocessorService not implemented"); - } - @Override public long getReadRpcTimeout(TimeUnit unit) { throw new UnsupportedOperationException(); @@ -910,8 +874,4 @@ public boolean thenMutate(RowMutations mutation) throws IOException { } } - @Override - public RegionLocator getRegionLocator() throws IOException { - throw new UnsupportedOperationException(); - } } diff --git a/pom.xml b/pom.xml index 904ca877..7570b329 100644 --- a/pom.xml +++ b/pom.xml @@ -59,11 +59,11 @@ - hbase-connectors-protocal-shade kafka spark hbase-rest hbase-connectors-assembly + hbase-connectors-protocol-shaded scm:git:git://gitbox.apache.org/repos/asf/hbase-connectors.git @@ -327,14 +327,10 @@ org.apache.hbase - hbase-mapreduce - ${hbase.version} - - - org.apache.hbase - hbase-protocol-shaded + hbase-http ${hbase.version} + org.apache.hbase.thirdparty hbase-shaded-protobuf