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 TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
058      TableName tableName, TableDescriptor currentDesc, TableDescriptor newDesc)
059      throws IOException {
060    verifyCoprocessors(ctx, newDesc);
061    return newDesc;
062  }
063
064  @Override
065  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
066      TableDescriptor htd, RegionInfo[] regions) throws IOException {
067    verifyCoprocessors(ctx, htd);
068  }
069
070  /**
071   * Validates a single whitelist path against the coprocessor path
072   * @param  coprocPath the path to the coprocessor including scheme
073   * @param  wlPath     can be:
074   *                      1) a "*" to wildcard all coprocessor paths
075   *                      2) a specific filesystem (e.g. hdfs://my-cluster/)
076   *                      3) a wildcard path to be evaluated by
077   *                         {@link FilenameUtils#wildcardMatch(String, String)}
078   *                         path can specify scheme or not (e.g.
079   *                         "file:///usr/hbase/coprocessors" or for all
080   *                         filesystems "/usr/hbase/coprocessors")
081   * @return             if the path was found under the wlPath
082   */
083  private static boolean validatePath(Path coprocPath, Path wlPath) {
084    // verify if all are allowed
085    if (wlPath.toString().equals("*")) {
086      return(true);
087    }
088
089    // verify we are on the same filesystem if wlPath has a scheme
090    if (!wlPath.isAbsoluteAndSchemeAuthorityNull()) {
091      String wlPathScheme = wlPath.toUri().getScheme();
092      String coprocPathScheme = coprocPath.toUri().getScheme();
093      String wlPathHost = wlPath.toUri().getHost();
094      String coprocPathHost = coprocPath.toUri().getHost();
095      if (wlPathScheme != null) {
096        wlPathScheme = wlPathScheme.toString().toLowerCase();
097      } else {
098        wlPathScheme = "";
099      }
100      if (wlPathHost != null) {
101        wlPathHost = wlPathHost.toString().toLowerCase();
102      } else {
103        wlPathHost = "";
104      }
105      if (coprocPathScheme != null) {
106        coprocPathScheme = coprocPathScheme.toString().toLowerCase();
107      } else {
108        coprocPathScheme = "";
109      }
110      if (coprocPathHost != null) {
111        coprocPathHost = coprocPathHost.toString().toLowerCase();
112      } else {
113        coprocPathHost = "";
114      }
115      if (!wlPathScheme.equals(coprocPathScheme) || !wlPathHost.equals(coprocPathHost)) {
116        return(false);
117      }
118    }
119
120    // allow any on this file-system (file systems were verified to be the same above)
121    if (wlPath.isRoot()) {
122      return(true);
123    }
124
125    // allow "loose" matches stripping scheme
126    if (FilenameUtils.wildcardMatch(
127        Path.getPathWithoutSchemeAndAuthority(coprocPath).toString(),
128        Path.getPathWithoutSchemeAndAuthority(wlPath).toString())) {
129      return(true);
130    }
131    return(false);
132  }
133
134  /**
135   * Perform the validation checks for a coprocessor to determine if the path
136   * is white listed or not.
137   * @throws IOException if path is not included in whitelist or a failure
138   *                     occurs in processing
139   * @param  ctx         as passed in from the coprocessor
140   * @param  htd         as passed in from the coprocessor
141   */
142  private static void verifyCoprocessors(ObserverContext<MasterCoprocessorEnvironment> ctx,
143      TableDescriptor htd) throws IOException {
144    Collection<String> paths =
145      ctx.getEnvironment().getConfiguration().getStringCollection(
146            CP_COPROCESSOR_WHITELIST_PATHS_KEY);
147    for (CoprocessorDescriptor cp : htd.getCoprocessorDescriptors()) {
148      if (cp.getJarPath().isPresent()) {
149        if (paths.stream().noneMatch(p -> {
150          Path wlPath = new Path(p);
151          if (validatePath(new Path(cp.getJarPath().get()), wlPath)) {
152            LOG.debug(String.format("Coprocessor %s found in directory %s",
153              cp.getClassName(), p));
154            return true;
155          }
156          return false;
157        })) {
158          throw new IOException(String.format("Loading %s DENIED in %s",
159            cp.getClassName(), CP_COPROCESSOR_WHITELIST_PATHS_KEY));
160        }
161      }
162    }
163  }
164}