forked from apache/hive
-
Notifications
You must be signed in to change notification settings - Fork 0
/
RangerRestClientImpl.java
481 lines (447 loc) · 21.8 KB
/
RangerRestClientImpl.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hive.ql.exec.repl.ranger;
import com.google.common.annotations.VisibleForTesting;
import com.google.gson.Gson;
import com.google.gson.GsonBuilder;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.config.DefaultClientConfig;
import com.sun.jersey.multipart.FormDataMultiPart;
import com.sun.jersey.multipart.MultiPart;
import com.sun.jersey.multipart.file.StreamDataBodyPart;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.utils.Retry;
import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
import org.apache.hadoop.hive.ql.exec.util.Retryable;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.http.client.utils.URIBuilder;
import org.eclipse.jetty.util.MultiPartWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServletResponse;
import javax.ws.rs.core.MediaType;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.File;
import java.io.InputStreamReader;
import java.io.InputStream;
import java.io.Reader;
import java.io.FileNotFoundException;
import java.net.URISyntaxException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Arrays;
import java.util.concurrent.Callable;
/**
* RangerRestClientImpl to connect to Ranger and export policies.
*/
public class RangerRestClientImpl implements RangerRestClient {
private static final Logger LOG = LoggerFactory.getLogger(RangerRestClientImpl.class);
private static final String RANGER_REST_URL_EXPORTJSONFILE = "service/plugins/policies/exportJson";
private static final String RANGER_REST_URL_IMPORTJSONFILE =
"service/plugins/policies/importPoliciesFromFile";
public RangerExportPolicyList exportRangerPolicies(String sourceRangerEndpoint,
String dbName,
String rangerHiveServiceName,
HiveConf hiveConf)throws SemanticException {
LOG.info("Ranger endpoint for cluster " + sourceRangerEndpoint);
if (StringUtils.isEmpty(rangerHiveServiceName)) {
throw new SemanticException(ErrorMsg.REPL_INVALID_CONFIG_FOR_SERVICE.format("Ranger Service Name " +
"cannot be empty", ReplUtils.REPL_RANGER_SERVICE));
}
Retryable retryable = Retryable.builder()
.withHiveConf(hiveConf)
.withRetryOnException(Exception.class).build();
try {
return retryable.executeCallable(() -> exportRangerPoliciesPlain(sourceRangerEndpoint, rangerHiveServiceName,
dbName));
} catch (Exception e) {
throw new SemanticException(ErrorMsg.REPL_RETRY_EXHAUSTED.format(e.getMessage()), e);
}
}
@VisibleForTesting
RangerExportPolicyList exportRangerPoliciesPlain(String sourceRangerEndpoint,
String rangerHiveServiceName,
String dbName) throws SemanticException, URISyntaxException {
String finalUrl = getRangerExportUrl(sourceRangerEndpoint, rangerHiveServiceName, dbName);
LOG.debug("Url to export policies from source Ranger: {}", finalUrl);
WebResource.Builder builder = getRangerResourceBuilder(finalUrl);
RangerExportPolicyList rangerExportPolicyList = new RangerExportPolicyList();
ClientResponse clientResp = builder.get(ClientResponse.class);
String response = null;
if (clientResp != null) {
if (clientResp.getStatus() == HttpServletResponse.SC_OK) {
Gson gson = new GsonBuilder().create();
response = clientResp.getEntity(String.class);
LOG.debug("Response received for ranger export {} ", response);
if (StringUtils.isNotEmpty(response)) {
rangerExportPolicyList = gson.fromJson(response, RangerExportPolicyList.class);
return rangerExportPolicyList;
}
} else if (clientResp.getStatus() == HttpServletResponse.SC_NO_CONTENT) {
LOG.debug("Ranger policy export request returned empty list");
return rangerExportPolicyList;
} else if (clientResp.getStatus() == HttpServletResponse.SC_UNAUTHORIZED) {
throw new SemanticException("Authentication Failure while communicating to Ranger admin");
} else if (clientResp.getStatus() == HttpServletResponse.SC_FORBIDDEN) {
throw new SemanticException("Authorization Failure while communicating to Ranger admin");
}
}
if (StringUtils.isEmpty(response)) {
LOG.debug("Ranger policy export request returned empty list or failed, Please refer Ranger admin logs.");
}
return null;
}
public String getRangerExportUrl(String sourceRangerEndpoint, String rangerHiveServiceName,
String dbName) throws URISyntaxException {
URIBuilder uriBuilder = new URIBuilder(sourceRangerEndpoint);
uriBuilder.setPath(RANGER_REST_URL_EXPORTJSONFILE);
uriBuilder.addParameter("serviceName", rangerHiveServiceName);
uriBuilder.addParameter("polResource", dbName);
uriBuilder.addParameter("resource:database", dbName);
uriBuilder.addParameter("serviceType", "hive");
uriBuilder.addParameter("resourceMatchScope", "self_or_ancestor");
uriBuilder.addParameter("resourceMatch", "full");
return uriBuilder.build().toString();
}
public List<RangerPolicy> removeMultiResourcePolicies(List<RangerPolicy> rangerPolicies) {
List<RangerPolicy> rangerPoliciesToImport = new ArrayList<RangerPolicy>();
if (CollectionUtils.isNotEmpty(rangerPolicies)) {
Map<String, RangerPolicy.RangerPolicyResource> rangerPolicyResourceMap = null;
RangerPolicy.RangerPolicyResource rangerPolicyResource = null;
List<String> resourceNameList = null;
for (RangerPolicy rangerPolicy : rangerPolicies) {
if (rangerPolicy != null) {
rangerPolicyResourceMap = rangerPolicy.getResources();
if (rangerPolicyResourceMap != null) {
rangerPolicyResource = rangerPolicyResourceMap.get("database");
if (rangerPolicyResource != null) {
resourceNameList = rangerPolicyResource.getValues();
if (CollectionUtils.isNotEmpty(resourceNameList) && resourceNameList.size() == 1) {
rangerPoliciesToImport.add(rangerPolicy);
}
}
}
}
}
}
return rangerPoliciesToImport;
}
@Override
public RangerExportPolicyList importRangerPolicies(RangerExportPolicyList rangerExportPolicyList, String dbName,
String baseUrl,
String rangerHiveServiceName,
HiveConf hiveConf)
throws Exception {
String sourceClusterServiceName = null;
String serviceMapJsonFileName = "hive_servicemap.json";
String rangerPoliciesJsonFileName = "hive_replicationPolicies.json";
if (!rangerExportPolicyList.getPolicies().isEmpty()) {
sourceClusterServiceName = rangerExportPolicyList.getPolicies().get(0).getService();
}
if (StringUtils.isEmpty(sourceClusterServiceName)) {
sourceClusterServiceName = rangerHiveServiceName;
}
Map<String, String> serviceMap = new LinkedHashMap<String, String>();
if (!StringUtils.isEmpty(sourceClusterServiceName) && !StringUtils.isEmpty(rangerHiveServiceName)) {
serviceMap.put(sourceClusterServiceName, rangerHiveServiceName);
}
Gson gson = new GsonBuilder().create();
String jsonServiceMap = gson.toJson(serviceMap);
String jsonRangerExportPolicyList = gson.toJson(rangerExportPolicyList);
String finalUrl = getRangerImportUrl(baseUrl, dbName);
LOG.debug("URL to import policies on target Ranger: {}", finalUrl);
Retryable retryable = Retryable.builder()
.withHiveConf(hiveConf)
.withRetryOnException(Exception.class).build();
return retryable.executeCallable(() -> importRangerPoliciesPlain(jsonRangerExportPolicyList,
rangerPoliciesJsonFileName,
serviceMapJsonFileName, jsonServiceMap, finalUrl, rangerExportPolicyList));
}
private RangerExportPolicyList importRangerPoliciesPlain(String jsonRangerExportPolicyList,
String rangerPoliciesJsonFileName,
String serviceMapJsonFileName, String jsonServiceMap,
String finalUrl, RangerExportPolicyList
rangerExportPolicyList) throws Exception {
ClientResponse clientResp = null;
StreamDataBodyPart filePartPolicies = new StreamDataBodyPart("file",
new ByteArrayInputStream(jsonRangerExportPolicyList.getBytes(StandardCharsets.UTF_8)),
rangerPoliciesJsonFileName);
StreamDataBodyPart filePartServiceMap = new StreamDataBodyPart("servicesMapJson",
new ByteArrayInputStream(jsonServiceMap.getBytes(StandardCharsets.UTF_8)), serviceMapJsonFileName);
FormDataMultiPart formDataMultiPart = new FormDataMultiPart();
MultiPart multipartEntity = null;
try {
multipartEntity = formDataMultiPart.bodyPart(filePartPolicies).bodyPart(filePartServiceMap);
WebResource.Builder builder = getRangerResourceBuilder(finalUrl);
clientResp = builder.accept(MediaType.APPLICATION_JSON).type(MediaType.MULTIPART_FORM_DATA)
.post(ClientResponse.class, multipartEntity);
if (clientResp != null) {
if (clientResp.getStatus() == HttpServletResponse.SC_NO_CONTENT) {
LOG.debug("Ranger policy import finished successfully");
} else if (clientResp.getStatus() == HttpServletResponse.SC_UNAUTHORIZED) {
throw new Exception("Authentication Failure while communicating to Ranger admin");
} else {
throw new Exception("Ranger policy import failed, Please refer target Ranger admin logs.");
}
}
} finally {
try {
if (filePartPolicies != null) {
filePartPolicies.cleanup();
}
if (filePartServiceMap != null) {
filePartServiceMap.cleanup();
}
if (formDataMultiPart != null) {
formDataMultiPart.close();
}
if (multipartEntity != null) {
multipartEntity.close();
}
} catch (IOException e) {
LOG.error("Exception occurred while closing resources: {}", e);
}
}
return rangerExportPolicyList;
}
public String getRangerImportUrl(String rangerUrl, String dbName) throws URISyntaxException {
URIBuilder uriBuilder = new URIBuilder(rangerUrl);
uriBuilder.setPath(RANGER_REST_URL_IMPORTJSONFILE);
uriBuilder.addParameter("mergeIfExists", "true");
uriBuilder.addParameter("polResource", dbName);
return uriBuilder.build().toString();
}
private synchronized Client getRangerClient() {
Client ret = null;
ClientConfig config = new DefaultClientConfig();
config.getClasses().add(MultiPartWriter.class);
config.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, true);
ret = Client.create(config);
return ret;
}
@Override
public List<RangerPolicy> changeDataSet(List<RangerPolicy> rangerPolicies, String sourceDbName,
String targetDbName) {
if (StringUtils.isEmpty(sourceDbName) || StringUtils.isEmpty(targetDbName) || targetDbName.equals(sourceDbName)) {
return rangerPolicies;
}
if (CollectionUtils.isNotEmpty(rangerPolicies)) {
Map<String, RangerPolicy.RangerPolicyResource> rangerPolicyResourceMap = null;
RangerPolicy.RangerPolicyResource rangerPolicyResource = null;
List<String> resourceNameList = null;
for (RangerPolicy rangerPolicy : rangerPolicies) {
if (rangerPolicy != null) {
rangerPolicyResourceMap = rangerPolicy.getResources();
if (rangerPolicyResourceMap != null) {
rangerPolicyResource = rangerPolicyResourceMap.get("database");
if (rangerPolicyResource != null) {
resourceNameList = rangerPolicyResource.getValues();
if (CollectionUtils.isNotEmpty(resourceNameList)) {
for (int i = 0; i < resourceNameList.size(); i++) {
String resourceName = resourceNameList.get(i);
if (resourceName.equals(sourceDbName)) {
resourceNameList.set(i, targetDbName);
}
}
}
}
}
}
}
}
return rangerPolicies;
}
private Path writeExportedRangerPoliciesToJsonFile(String jsonString, String fileName, Path stagingDirPath,
HiveConf conf)
throws IOException {
String filePath = "";
Path newPath = null;
FSDataOutputStream outStream = null;
OutputStreamWriter writer = null;
try {
if (!StringUtils.isEmpty(jsonString)) {
FileSystem fileSystem = stagingDirPath.getFileSystem(conf);
if (fileSystem != null) {
if (!fileSystem.exists(stagingDirPath)) {
fileSystem.mkdirs(stagingDirPath);
}
newPath = stagingDirPath.suffix(File.separator + fileName);
outStream = fileSystem.create(newPath, true);
writer = new OutputStreamWriter(outStream, "UTF-8");
writer.write(jsonString);
}
}
} catch (IOException ex) {
if (newPath != null) {
filePath = newPath.toString();
}
throw new IOException("Failed to write json string to file:" + filePath, ex);
} catch (Exception ex) {
if (newPath != null) {
filePath = newPath.toString();
}
throw new IOException("Failed to write json string to file:" + filePath, ex);
} finally {
try {
if (writer != null) {
writer.close();
}
if (outStream != null) {
outStream.close();
}
} catch (Exception ex) {
throw new IOException("Unable to close writer/outStream.", ex);
}
}
return newPath;
}
@Override
public Path saveRangerPoliciesToFile(RangerExportPolicyList rangerExportPolicyList, Path stagingDirPath,
String fileName, HiveConf conf) throws SemanticException {
Gson gson = new GsonBuilder().create();
String jsonRangerExportPolicyList = gson.toJson(rangerExportPolicyList);
Retryable retryable = Retryable.builder()
.withHiveConf(conf)
.withRetryOnException(IOException.class).build();
try {
return retryable.executeCallable(() -> writeExportedRangerPoliciesToJsonFile(jsonRangerExportPolicyList, fileName,
stagingDirPath, conf));
} catch (Exception e) {
throw new SemanticException(ErrorMsg.REPL_RETRY_EXHAUSTED.format(e.getMessage()), e);
}
}
@Override
public RangerExportPolicyList readRangerPoliciesFromJsonFile(Path filePath,
HiveConf conf) throws SemanticException {
RangerExportPolicyList rangerExportPolicyList = null;
Gson gsonBuilder = new GsonBuilder().setDateFormat("yyyyMMdd-HH:mm:ss.SSS-Z").setPrettyPrinting().create();
try {
FileSystem fs = filePath.getFileSystem(conf);
InputStream inputStream = fs.open(filePath);
Reader reader = new InputStreamReader(inputStream, Charset.forName("UTF-8"));
rangerExportPolicyList = gsonBuilder.fromJson(reader, RangerExportPolicyList.class);
} catch (FileNotFoundException e) {
//If the ranger policies are not present, json file will not be present
return rangerExportPolicyList;
} catch (Exception ex) {
throw new SemanticException("Error reading file :" + filePath, ex);
}
return rangerExportPolicyList;
}
@Override
public boolean checkConnection(String url, HiveConf hiveConf) throws SemanticException {
Retryable retryable = Retryable.builder()
.withHiveConf(hiveConf)
.withRetryOnException(Exception.class).build();
try {
return retryable.executeCallable(() -> checkConnectionPlain(url));
} catch (Exception e) {
throw new SemanticException(ErrorMsg.REPL_RETRY_EXHAUSTED.format(e.getMessage()), e);
}
}
@VisibleForTesting
boolean checkConnectionPlain(String url) {
WebResource.Builder builder;
builder = getRangerResourceBuilder(url);
ClientResponse clientResp = builder.get(ClientResponse.class);
return (clientResp.getStatus() < HttpServletResponse.SC_UNAUTHORIZED);
}
@Override
public List<RangerPolicy> addDenyPolicies(List<RangerPolicy> rangerPolicies, String rangerServiceName,
String sourceDb, String targetDb) throws SemanticException {
if (StringUtils.isEmpty(rangerServiceName)) {
throw new SemanticException(ErrorMsg.REPL_INVALID_CONFIG_FOR_SERVICE.format("Ranger Service " +
"Name cannot be empty", ReplUtils.REPL_RANGER_SERVICE));
}
RangerPolicy denyRangerPolicy = new RangerPolicy();
denyRangerPolicy.setService(rangerServiceName);
denyRangerPolicy.setName(sourceDb + "_replication deny policy for " + targetDb);
Map<String, RangerPolicy.RangerPolicyResource> rangerPolicyResourceMap = new HashMap<String,
RangerPolicy.RangerPolicyResource>();
RangerPolicy.RangerPolicyResource rangerPolicyResource = new RangerPolicy.RangerPolicyResource();
List<String> resourceNameList = new ArrayList<String>();
List<RangerPolicy.RangerPolicyItem> denyPolicyItemsForPublicGroup = denyRangerPolicy.getDenyPolicyItems();
RangerPolicy.RangerPolicyItem denyPolicyItem = new RangerPolicy.RangerPolicyItem();
List<RangerPolicy.RangerPolicyItemAccess> denyPolicyItemAccesses = new ArrayList<RangerPolicy.
RangerPolicyItemAccess>();
List<RangerPolicy.RangerPolicyItem> denyExceptionsItemsForBeaconUser = denyRangerPolicy.getDenyExceptions();
RangerPolicy.RangerPolicyItem denyExceptionsPolicyItem = new RangerPolicy.RangerPolicyItem();
List<RangerPolicy.RangerPolicyItemAccess> denyExceptionsPolicyItemAccesses = new ArrayList<RangerPolicy.
RangerPolicyItemAccess>();
resourceNameList.add(sourceDb);
rangerPolicyResource.setValues(resourceNameList);
RangerPolicy.RangerPolicyResource rangerPolicyResourceColumn =new RangerPolicy.RangerPolicyResource();
rangerPolicyResourceColumn.setValues(new ArrayList<String>(){{add("*"); }});
RangerPolicy.RangerPolicyResource rangerPolicyResourceTable =new RangerPolicy.RangerPolicyResource();
rangerPolicyResourceTable.setValues(new ArrayList<String>(){{add("*"); }});
rangerPolicyResourceMap.put("database", rangerPolicyResource);
rangerPolicyResourceMap.put("table", rangerPolicyResourceTable);
rangerPolicyResourceMap.put("column", rangerPolicyResourceColumn);
denyRangerPolicy.setResources(rangerPolicyResourceMap);
List<String> accessTypes = Arrays.asList("create", "update", "drop", "alter",
"index", "lock", "write", "ReplAdmin");
for (String access : accessTypes) {
denyPolicyItemAccesses.add(new RangerPolicy.RangerPolicyItemAccess(access, true));
}
denyPolicyItem.setAccesses(denyPolicyItemAccesses);
denyPolicyItemsForPublicGroup.add(denyPolicyItem);
List<String> denyPolicyItemsGroups = new ArrayList<String>();
denyPolicyItemsGroups.add("public");
denyPolicyItem.setGroups(denyPolicyItemsGroups);
denyRangerPolicy.setDenyPolicyItems(denyPolicyItemsForPublicGroup);
List<String> denyExcludeAccessTypes = Arrays.asList("create", "update", "drop", "alter", "index", "lock", "write",
"ReplAdmin", "select", "read");
for (String access : denyExcludeAccessTypes) {
denyExceptionsPolicyItemAccesses.add(new RangerPolicy.RangerPolicyItemAccess(access, true));
}
denyExceptionsPolicyItem.setAccesses(denyExceptionsPolicyItemAccesses);
denyExceptionsItemsForBeaconUser.add(denyExceptionsPolicyItem);
List<String> denyExceptionsPolicyItemsUsers = new ArrayList<String>();
denyExceptionsPolicyItemsUsers.add("hive");
denyExceptionsPolicyItem.setUsers(denyExceptionsPolicyItemsUsers);
denyRangerPolicy.setDenyExceptions(denyExceptionsItemsForBeaconUser);
rangerPolicies.add(denyRangerPolicy);
return rangerPolicies;
}
private WebResource.Builder getRangerResourceBuilder(String url) {
Client client = getRangerClient();
WebResource webResource = client.resource(url);
WebResource.Builder builder = webResource.getRequestBuilder();
return builder;
}
}