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 com.google.protobuf.RpcCallback;
022import com.google.protobuf.RpcController;
023import com.google.protobuf.Service;
024
025import java.io.IOException;
026import java.util.Collections;
027import java.util.List;
028import java.util.Map;
029
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.CoprocessorEnvironment;
032import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
033import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
034import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
035import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
036import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
037import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
038import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
039import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
040import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
041import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
042import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
043import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
044import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
045import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
046import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
047import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
048import org.apache.hadoop.hbase.regionserver.HRegion;
049import org.apache.hadoop.hbase.regionserver.RegionServerServices;
050import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
051
052import org.apache.yetus.audience.InterfaceAudience;
053import org.slf4j.Logger;
054import org.slf4j.LoggerFactory;
055
056/**
057 * Coprocessor service for bulk loads in secure mode.
058 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
059 */
060@CoreCoprocessor
061@InterfaceAudience.Private
062@Deprecated
063public class SecureBulkLoadEndpoint extends SecureBulkLoadService implements RegionCoprocessor {
064  public static final long VERSION = 0L;
065
066  private static final Logger LOG = LoggerFactory.getLogger(SecureBulkLoadEndpoint.class);
067
068  private RegionCoprocessorEnvironment env;
069  private RegionServerServices rsServices;
070
071  @Override
072  public void start(CoprocessorEnvironment env) {
073    this.env = (RegionCoprocessorEnvironment)env;
074    rsServices = ((HasRegionServerServices)this.env).getRegionServerServices();
075    LOG.warn("SecureBulkLoadEndpoint is deprecated. It will be removed in future releases.");
076    LOG.warn("Secure bulk load has been integrated into HBase core.");
077  }
078
079  @Override
080  public void stop(CoprocessorEnvironment env) throws IOException {
081  }
082
083  @Override
084  public void prepareBulkLoad(RpcController controller, PrepareBulkLoadRequest request,
085          RpcCallback<PrepareBulkLoadResponse> done) {
086    try {
087      SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
088
089      String bulkToken = secureBulkLoadManager.prepareBulkLoad((HRegion) this.env.getRegion(),
090          convert(request));
091      done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
092    } catch (IOException e) {
093      CoprocessorRpcUtils.setControllerException(controller, e);
094    }
095    done.run(null);
096  }
097
098  /**
099   *  Convert from CPEP protobuf 2.5 to internal protobuf 3.3.
100   */
101  org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest
102    convert(PrepareBulkLoadRequest request)
103    throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException {
104    byte [] bytes = request.toByteArray();
105    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest.Builder
106          builder =
107        org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest.
108        newBuilder();
109    builder.mergeFrom(bytes);
110    return builder.build();
111  }
112
113  @Override
114  public void cleanupBulkLoad(RpcController controller, CleanupBulkLoadRequest request,
115          RpcCallback<CleanupBulkLoadResponse> done) {
116    try {
117      SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
118      secureBulkLoadManager.cleanupBulkLoad((HRegion) this.env.getRegion(), convert(request));
119      done.run(CleanupBulkLoadResponse.newBuilder().build());
120    } catch (IOException e) {
121      CoprocessorRpcUtils.setControllerException(controller, e);
122    }
123    done.run(null);
124  }
125
126  /**
127   *  Convert from CPEP protobuf 2.5 to internal protobuf 3.3.
128   */
129  org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest
130    convert(CleanupBulkLoadRequest request)
131      throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException {
132    byte [] bytes = request.toByteArray();
133    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest.Builder
134        builder =
135      org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest.
136      newBuilder();
137    builder.mergeFrom(bytes);
138    return builder.build();
139  }
140
141  @Override
142  public void secureBulkLoadHFiles(RpcController controller, SecureBulkLoadHFilesRequest request,
143          RpcCallback<SecureBulkLoadHFilesResponse> done) {
144    boolean loaded = false;
145    Map<byte[], List<Path>> map = null;
146    try {
147      SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
148      BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request);
149      map = secureBulkLoadManager.secureBulkLoadHFiles((HRegion) this.env.getRegion(),
150          convert(bulkLoadHFileRequest));
151      loaded = map != null && !map.isEmpty();
152    } catch (IOException e) {
153      CoprocessorRpcUtils.setControllerException(controller, e);
154    }
155    done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
156  }
157
158  /**
159   *  Convert from CPEP protobuf 2.5 to internal protobuf 3.3.
160   */
161  org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest
162    convert(BulkLoadHFileRequest request)
163      throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException {
164    byte [] bytes = request.toByteArray();
165    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.Builder
166        builder =
167      org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.
168        newBuilder();
169    builder.mergeFrom(bytes);
170    return builder.build();
171  }
172
173  private BulkLoadHFileRequest ConvertSecureBulkLoadHFilesRequest(
174          SecureBulkLoadHFilesRequest request) {
175    BulkLoadHFileRequest.Builder bulkLoadHFileRequest = BulkLoadHFileRequest.newBuilder();
176    RegionSpecifier region =
177        ProtobufUtil.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, this.env
178            .getRegionInfo().getRegionName());
179    bulkLoadHFileRequest.setRegion(region).setFsToken(request.getFsToken())
180        .setBulkToken(request.getBulkToken()).setAssignSeqNum(request.getAssignSeqNum())
181        .addAllFamilyPath(request.getFamilyPathList());
182    return bulkLoadHFileRequest.build();
183  }
184
185  @Override
186  public Iterable<Service> getServices() {
187    return Collections.singleton(this);
188  }
189}