001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.security.access;
020
021import java.io.IOException;
022import java.util.Collection;
023import java.util.Optional;
024import org.apache.commons.io.FilenameUtils;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.HBaseInterfaceAudience;
027import org.apache.hadoop.hbase.TableName;
028import org.apache.hadoop.hbase.client.CoprocessorDescriptor;
029import org.apache.hadoop.hbase.client.RegionInfo;
030import org.apache.hadoop.hbase.client.TableDescriptor;
031import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
032import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
033import org.apache.hadoop.hbase.coprocessor.MasterObserver;
034import org.apache.hadoop.hbase.coprocessor.ObserverContext;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039/**
040 * Master observer for restricting coprocessor assignments.
041 */
042@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
043public class CoprocessorWhitelistMasterObserver implements MasterCoprocessor, MasterObserver {
044
045  public static final String CP_COPROCESSOR_WHITELIST_PATHS_KEY =
046      "hbase.coprocessor.region.whitelist.paths";
047
048  private static final Logger LOG = LoggerFactory
049      .getLogger(CoprocessorWhitelistMasterObserver.class);
050
051  @Override
052  public Optional<MasterObserver> getMasterObserver() {
053    return Optional.of(this);
054  }
055
056  @Override
057  public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
058      TableName tableName, TableDescriptor htd) throws IOException {
059    verifyCoprocessors(ctx, htd);
060  }
061
062  @Override
063  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
064      TableDescriptor htd, RegionInfo[] regions) throws IOException {
065    verifyCoprocessors(ctx, htd);
066  }
067
068  /**
069   * Validates a single whitelist path against the coprocessor path
070   * @param  coprocPath the path to the coprocessor including scheme
071   * @param  wlPath     can be:
072   *                      1) a "*" to wildcard all coprocessor paths
073   *                      2) a specific filesystem (e.g. hdfs://my-cluster/)
074   *                      3) a wildcard path to be evaluated by
075   *                         {@link FilenameUtils#wildcardMatch(String, String)}
076   *                         path can specify scheme or not (e.g.
077   *                         "file:///usr/hbase/coprocessors" or for all
078   *                         filesystems "/usr/hbase/coprocessors")
079   * @return             if the path was found under the wlPath
080   */
081  private static boolean validatePath(Path coprocPath, Path wlPath) {
082    // verify if all are allowed
083    if (wlPath.toString().equals("*")) {
084      return(true);
085    }
086
087    // verify we are on the same filesystem if wlPath has a scheme
088    if (!wlPath.isAbsoluteAndSchemeAuthorityNull()) {
089      String wlPathScheme = wlPath.toUri().getScheme();
090      String coprocPathScheme = coprocPath.toUri().getScheme();
091      String wlPathHost = wlPath.toUri().getHost();
092      String coprocPathHost = coprocPath.toUri().getHost();
093      if (wlPathScheme != null) {
094        wlPathScheme = wlPathScheme.toString().toLowerCase();
095      } else {
096        wlPathScheme = "";
097      }
098      if (wlPathHost != null) {
099        wlPathHost = wlPathHost.toString().toLowerCase();
100      } else {
101        wlPathHost = "";
102      }
103      if (coprocPathScheme != null) {
104        coprocPathScheme = coprocPathScheme.toString().toLowerCase();
105      } else {
106        coprocPathScheme = "";
107      }
108      if (coprocPathHost != null) {
109        coprocPathHost = coprocPathHost.toString().toLowerCase();
110      } else {
111        coprocPathHost = "";
112      }
113      if (!wlPathScheme.equals(coprocPathScheme) || !wlPathHost.equals(coprocPathHost)) {
114        return(false);
115      }
116    }
117
118    // allow any on this file-system (file systems were verified to be the same above)
119    if (wlPath.isRoot()) {
120      return(true);
121    }
122
123    // allow "loose" matches stripping scheme
124    if (FilenameUtils.wildcardMatch(
125        Path.getPathWithoutSchemeAndAuthority(coprocPath).toString(),
126        Path.getPathWithoutSchemeAndAuthority(wlPath).toString())) {
127      return(true);
128    }
129    return(false);
130  }
131
132  /**
133   * Perform the validation checks for a coprocessor to determine if the path
134   * is white listed or not.
135   * @throws IOException if path is not included in whitelist or a failure
136   *                     occurs in processing
137   * @param  ctx         as passed in from the coprocessor
138   * @param  htd         as passed in from the coprocessor
139   */
140  private static void verifyCoprocessors(ObserverContext<MasterCoprocessorEnvironment> ctx,
141      TableDescriptor htd) throws IOException {
142    Collection<String> paths =
143      ctx.getEnvironment().getConfiguration().getStringCollection(
144            CP_COPROCESSOR_WHITELIST_PATHS_KEY);
145    for (CoprocessorDescriptor cp : htd.getCoprocessorDescriptors()) {
146      if (cp.getJarPath().isPresent()) {
147        if (paths.stream().noneMatch(p -> {
148          Path wlPath = new Path(p);
149          if (validatePath(new Path(cp.getJarPath().get()), wlPath)) {
150            LOG.debug(String.format("Coprocessor %s found in directory %s",
151              cp.getClassName(), p));
152            return true;
153          }
154          return false;
155        })) {
156          throw new IOException(String.format("Loading %s DENIED in %s",
157            cp.getClassName(), CP_COPROCESSOR_WHITELIST_PATHS_KEY));
158        }
159      }
160    }
161  }
162}