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 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest 099 convert(PrepareBulkLoadRequest request) 100 throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException { 101 byte [] bytes = request.toByteArray(); 102 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest.Builder 103 builder = 104 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest. 105 newBuilder(); 106 builder.mergeFrom(bytes); 107 return builder.build(); 108 } 109 110 @Override 111 public void cleanupBulkLoad(RpcController controller, CleanupBulkLoadRequest request, 112 RpcCallback<CleanupBulkLoadResponse> done) { 113 try { 114 SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager(); 115 secureBulkLoadManager.cleanupBulkLoad((HRegion) this.env.getRegion(), convert(request)); 116 done.run(CleanupBulkLoadResponse.newBuilder().build()); 117 } catch (IOException e) { 118 CoprocessorRpcUtils.setControllerException(controller, e); 119 } 120 done.run(null); 121 } 122 123 /** 124 * Convert from CPEP protobuf 2.5 to internal protobuf 3.3. 125 * @throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException 126 */ 127 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest 128 convert(CleanupBulkLoadRequest request) 129 throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException { 130 byte [] bytes = request.toByteArray(); 131 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest.Builder 132 builder = 133 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest. 134 newBuilder(); 135 builder.mergeFrom(bytes); 136 return builder.build(); 137 } 138 139 @Override 140 public void secureBulkLoadHFiles(RpcController controller, SecureBulkLoadHFilesRequest request, 141 RpcCallback<SecureBulkLoadHFilesResponse> done) { 142 boolean loaded = false; 143 Map<byte[], List<Path>> map = null; 144 try { 145 SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager(); 146 BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request); 147 map = secureBulkLoadManager.secureBulkLoadHFiles((HRegion) this.env.getRegion(), 148 convert(bulkLoadHFileRequest)); 149 loaded = map != null && !map.isEmpty(); 150 } catch (IOException e) { 151 CoprocessorRpcUtils.setControllerException(controller, e); 152 } 153 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build()); 154 } 155 156 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest 157 convert(BulkLoadHFileRequest request) 158 throws org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException { 159 byte [] bytes = request.toByteArray(); 160 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.Builder 161 builder = 162 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest. 163 newBuilder(); 164 builder.mergeFrom(bytes); 165 return builder.build(); 166 } 167 168 private BulkLoadHFileRequest ConvertSecureBulkLoadHFilesRequest( 169 SecureBulkLoadHFilesRequest request) { 170 BulkLoadHFileRequest.Builder bulkLoadHFileRequest = BulkLoadHFileRequest.newBuilder(); 171 RegionSpecifier region = 172 ProtobufUtil.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, this.env 173 .getRegionInfo().getRegionName()); 174 bulkLoadHFileRequest.setRegion(region).setFsToken(request.getFsToken()) 175 .setBulkToken(request.getBulkToken()).setAssignSeqNum(request.getAssignSeqNum()) 176 .addAllFamilyPath(request.getFamilyPathList()); 177 return bulkLoadHFileRequest.build(); 178 } 179 180 @Override 181 public Iterable<Service> getServices() { 182 return Collections.singleton(this); 183 } 184}