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 */ 018package org.apache.hadoop.hbase.thrift; 019 020import java.io.File; 021import java.nio.ByteBuffer; 022import java.security.Principal; 023import java.security.PrivilegedExceptionAction; 024import java.util.ArrayList; 025import java.util.Base64; 026import java.util.HashMap; 027import java.util.HashSet; 028import java.util.Map; 029import java.util.Set; 030import javax.security.auth.Subject; 031import javax.security.auth.kerberos.KerberosPrincipal; 032import javax.security.auth.login.AppConfigurationEntry; 033import javax.security.auth.login.Configuration; 034import javax.security.auth.login.LoginContext; 035import org.apache.hadoop.hbase.thrift.generated.AlreadyExists; 036import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor; 037import org.apache.hadoop.hbase.thrift.generated.Hbase; 038import org.apache.hadoop.hbase.util.Bytes; 039import org.apache.hadoop.hbase.util.ClientUtils; 040import org.apache.thrift.protocol.TBinaryProtocol; 041import org.apache.thrift.protocol.TProtocol; 042import org.apache.thrift.transport.THttpClient; 043import org.apache.thrift.transport.TSocket; 044import org.apache.thrift.transport.TTransport; 045import org.apache.yetus.audience.InterfaceAudience; 046import org.ietf.jgss.GSSContext; 047import org.ietf.jgss.GSSCredential; 048import org.ietf.jgss.GSSException; 049import org.ietf.jgss.GSSManager; 050import org.ietf.jgss.GSSName; 051import org.ietf.jgss.Oid; 052import org.slf4j.Logger; 053import org.slf4j.LoggerFactory; 054 055/** 056 * See the instructions under hbase-examples/README.txt 057 */ 058@InterfaceAudience.Private 059public class HttpDoAsClient { 060 private static final Logger LOG = LoggerFactory.getLogger(HttpDoAsClient.class); 061 062 static protected int port; 063 static protected String host; 064 private static boolean secure = false; 065 static protected String doAsUser = null; 066 static protected String principal = null; 067 static protected String keyTab = null; 068 069 public static void main(String[] args) throws Exception { 070 if (args.length < 3 || args.length > 6) { 071 System.out.println("Invalid arguments!"); 072 System.out.println( 073 "Usage: HttpDoAsClient host port doAsUserName [security=true] [principal] [keytab]"); 074 System.exit(-1); 075 } 076 077 host = args[0]; 078 port = Integer.parseInt(args[1]); 079 doAsUser = args[2]; 080 if (args.length > 3) { 081 secure = Boolean.parseBoolean(args[3]); 082 if (args.length > 4) { 083 principal = args[4]; 084 keyTab = args[5]; 085 if (!new File(keyTab).exists()) { 086 System.err.printf("ERROR: KeyTab File %s not found %n", keyTab); 087 System.exit(-1); 088 } 089 } else { 090 principal = getSubject().getPrincipals().iterator().next().getName(); 091 } 092 } 093 094 final HttpDoAsClient client = new HttpDoAsClient(); 095 Subject.doAs(getSubject(), new PrivilegedExceptionAction<Void>() { 096 @Override 097 public Void run() throws Exception { 098 client.run(); 099 return null; 100 } 101 }); 102 } 103 104 HttpDoAsClient() { 105 } 106 107 // Helper to translate strings to UTF8 bytes 108 private byte[] bytes(String s) { 109 return Bytes.toBytes(s); 110 } 111 112 private void run() throws Exception { 113 TTransport transport = new TSocket(host, port); 114 115 transport.open(); 116 String url = "http://" + host + ":" + port; 117 THttpClient httpClient = new THttpClient(url); 118 httpClient.open(); 119 TProtocol protocol = new TBinaryProtocol(httpClient); 120 Hbase.Client client = new Hbase.Client(protocol); 121 122 byte[] t = bytes("demo_table"); 123 124 // 125 // Scan all tables, look for the demo table and delete it. 126 // 127 System.out.println("scanning tables..."); 128 for (ByteBuffer name : refresh(client, httpClient).getTableNames()) { 129 System.out.println(" found: " + ClientUtils.utf8(name)); 130 if (ClientUtils.utf8(name).equals(ClientUtils.utf8(t))) { 131 if (refresh(client, httpClient).isTableEnabled(name)) { 132 System.out.println(" disabling table: " + ClientUtils.utf8(name)); 133 refresh(client, httpClient).disableTable(name); 134 } 135 System.out.println(" deleting table: " + ClientUtils.utf8(name)); 136 refresh(client, httpClient).deleteTable(name); 137 } 138 } 139 140 // 141 // Create the demo table with two column families, entry: and unused: 142 // 143 ArrayList<ColumnDescriptor> columns = new ArrayList<>(2); 144 ColumnDescriptor col; 145 col = new ColumnDescriptor(); 146 col.name = ByteBuffer.wrap(bytes("entry:")); 147 col.timeToLive = Integer.MAX_VALUE; 148 col.maxVersions = 10; 149 columns.add(col); 150 col = new ColumnDescriptor(); 151 col.name = ByteBuffer.wrap(bytes("unused:")); 152 col.timeToLive = Integer.MAX_VALUE; 153 columns.add(col); 154 155 System.out.println("creating table: " + ClientUtils.utf8(t)); 156 try { 157 158 refresh(client, httpClient).createTable(ByteBuffer.wrap(t), columns); 159 } catch (AlreadyExists ae) { 160 System.out.println("WARN: " + ae.message); 161 } 162 163 System.out.println("column families in " + ClientUtils.utf8(t) + ": "); 164 Map<ByteBuffer, ColumnDescriptor> columnMap = 165 refresh(client, httpClient).getColumnDescriptors(ByteBuffer.wrap(t)); 166 for (ColumnDescriptor col2 : columnMap.values()) { 167 System.out 168 .println(" column: " + ClientUtils.utf8(col2.name) + ", maxVer: " + col2.maxVersions); 169 } 170 171 transport.close(); 172 httpClient.close(); 173 } 174 175 private Hbase.Client refresh(Hbase.Client client, THttpClient httpClient) { 176 httpClient.setCustomHeader("doAs", doAsUser); 177 if (secure) { 178 try { 179 httpClient.setCustomHeader("Authorization", generateTicket()); 180 } catch (GSSException e) { 181 LOG.error("Kerberos authentication failed", e); 182 } 183 } 184 return client; 185 } 186 187 private String generateTicket() throws GSSException { 188 final GSSManager manager = GSSManager.getInstance(); 189 // Oid for kerberos principal name 190 Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); 191 Oid KERB_V5_OID = new Oid("1.2.840.113554.1.2.2"); 192 final GSSName clientName = manager.createName(principal, krb5PrincipalOid); 193 final GSSCredential clientCred = 194 manager.createCredential(clientName, 8 * 3600, KERB_V5_OID, GSSCredential.INITIATE_ONLY); 195 196 final GSSName serverName = manager.createName(principal, krb5PrincipalOid); 197 198 final GSSContext context = 199 manager.createContext(serverName, KERB_V5_OID, clientCred, GSSContext.DEFAULT_LIFETIME); 200 context.requestMutualAuth(true); 201 context.requestConf(false); 202 context.requestInteg(true); 203 204 final byte[] outToken = context.initSecContext(new byte[0], 0, 0); 205 StringBuilder outputBuffer = new StringBuilder(); 206 outputBuffer.append("Negotiate "); 207 outputBuffer.append(Bytes.toString(Base64.getEncoder().encode(outToken))); 208 System.out.print("Ticket is: " + outputBuffer); 209 return outputBuffer.toString(); 210 } 211 212 static Subject getSubject() throws Exception { 213 if (!secure) { 214 return new Subject(); 215 } 216 217 /* 218 * To authenticate the DemoClient, kinit should be invoked ahead. Here we try to get the 219 * Kerberos credential from the ticket cache. 220 */ 221 LoginContext context; 222 223 if (keyTab != null) { 224 // To authenticate the HttpDoAsClient using principal and keyTab 225 Set<Principal> principals = new HashSet<>(); 226 principals.add(new KerberosPrincipal(principal)); 227 Subject subject = new Subject(false, principals, new HashSet<>(), new HashSet<>()); 228 229 context = new LoginContext("", subject, null, new KerberosConfiguration(principal, keyTab)); 230 } else { 231 /* 232 * To authenticate the HttpDoAsClient, kinit should be invoked ahead. Here we try to get the 233 * Kerberos credential from the ticket cache. 234 */ 235 context = new LoginContext("", new Subject(), null, new KerberosConfiguration()); 236 } 237 context.login(); 238 return context.getSubject(); 239 } 240 241 private static class KerberosConfiguration extends Configuration { 242 private String principal; 243 private String keyTab; 244 245 public KerberosConfiguration() { 246 // Empty constructor will have no principal or keyTab values 247 } 248 249 public KerberosConfiguration(String principal, String keyTab) { 250 this.principal = principal; 251 this.keyTab = keyTab; 252 } 253 254 @Override 255 public AppConfigurationEntry[] getAppConfigurationEntry(String name) { 256 Map<String, String> options = new HashMap<>(); 257 if (principal != null && keyTab != null) { 258 options.put("principal", principal); 259 options.put("keyTab", keyTab); 260 options.put("useKeyTab", "true"); 261 options.put("storeKey", "true"); 262 } else { 263 options.put("useKeyTab", "false"); 264 options.put("storeKey", "false"); 265 } 266 options.put("doNotPrompt", "true"); 267 options.put("useTicketCache", "true"); 268 options.put("renewTGT", "true"); 269 options.put("refreshKrb5Config", "true"); 270 options.put("isInitiator", "true"); 271 String ticketCache = System.getenv("KRB5CCNAME"); 272 if (ticketCache != null) { 273 options.put("ticketCache", ticketCache); 274 } 275 options.put("debug", "true"); 276 277 return new AppConfigurationEntry[] { 278 new AppConfigurationEntry("com.sun.security.auth.module.Krb5LoginModule", 279 AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options) }; 280 } 281 } 282}