/
HdfsFetcher.java
659 lines (561 loc) · 27.4 KB
/
HdfsFetcher.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
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
/*
* Copyright 2008-2009 LinkedIn, Inc
*
* Licensed 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 voldemort.store.readonly.fetcher;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.text.NumberFormat;
import java.util.Arrays;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicInteger;
import javax.management.ObjectName;
import org.apache.commons.codec.DecoderException;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.annotations.jmx.JmxGetter;
import voldemort.server.VoldemortConfig;
import voldemort.server.protocol.admin.AsyncOperationStatus;
import voldemort.store.readonly.FileFetcher;
import voldemort.store.readonly.ReadOnlyStorageMetadata;
import voldemort.store.readonly.checksum.CheckSum;
import voldemort.store.readonly.checksum.CheckSum.CheckSumType;
import voldemort.utils.ByteUtils;
import voldemort.utils.DynamicEventThrottler;
import voldemort.utils.DynamicThrottleLimit;
import voldemort.utils.EventThrottler;
import voldemort.utils.JmxUtils;
import voldemort.utils.Time;
import voldemort.utils.Utils;
/*
* A fetcher that fetches the store files from HDFS
*/
public class HdfsFetcher implements FileFetcher {
private static final Logger logger = Logger.getLogger(HdfsFetcher.class);
private static String keytabPath = "";
private static String kerberosPrincipal = "voldemrt";
private final Long maxBytesPerSecond, reportingIntervalBytes;
private final int bufferSize;
private static final AtomicInteger copyCount = new AtomicInteger(0);
private AsyncOperationStatus status;
private EventThrottler throttler = null;
private long minBytesPerSecond = 0;
private DynamicThrottleLimit globalThrottleLimit = null;
private static final int NUM_RETRIES = 3;
private String keytabLocation = "";
private String kerberosUser = "voldemrt";
private VoldemortConfig voldemortConfig = null;
public static final String FS_DEFAULT_NAME = "fs.default.name";
public HdfsFetcher(VoldemortConfig config, DynamicThrottleLimit dynThrottleLimit) {
this(dynThrottleLimit,
null,
config.getReportingIntervalBytes(),
config.getFetcherBufferSize(),
config.getMinBytesPerSecond(),
config.getReadOnlyKeytabPath(),
config.getReadOnlyKerberosUser());
this.voldemortConfig = config;
logger.info("Created hdfs fetcher with throttle rate " + dynThrottleLimit.getRate()
+ ", buffer size " + bufferSize + ", reporting interval bytes "
+ reportingIntervalBytes);
}
public HdfsFetcher() {
this((Long) null,
VoldemortConfig.REPORTING_INTERVAL_BYTES,
VoldemortConfig.DEFAULT_BUFFER_SIZE);
}
public HdfsFetcher(Long maxBytesPerSecond, Long reportingIntervalBytes, int bufferSize) {
this(null, maxBytesPerSecond, reportingIntervalBytes, bufferSize, 0, "", "");
}
public HdfsFetcher(DynamicThrottleLimit dynThrottleLimit,
Long maxBytesPerSecond,
Long reportingIntervalBytes,
int bufferSize,
long minBytesPerSecond,
String keytabLocation,
String kerberosUser) {
if(maxBytesPerSecond != null) {
this.maxBytesPerSecond = maxBytesPerSecond;
this.throttler = new EventThrottler(this.maxBytesPerSecond);
} else if(dynThrottleLimit != null && dynThrottleLimit.getRate() != 0) {
this.maxBytesPerSecond = dynThrottleLimit.getRate();
this.throttler = new DynamicEventThrottler(dynThrottleLimit);
this.globalThrottleLimit = dynThrottleLimit;
logger.info("Initializing Dynamic Event throttler with rate : "
+ this.maxBytesPerSecond + " bytes / sec");
} else
this.maxBytesPerSecond = null;
this.reportingIntervalBytes = Utils.notNull(reportingIntervalBytes);
this.bufferSize = bufferSize;
this.status = null;
this.minBytesPerSecond = minBytesPerSecond;
HdfsFetcher.kerberosPrincipal = kerberosUser;
HdfsFetcher.keytabPath = keytabLocation;
}
public File fetch(String sourceFileUrl, String destinationFile) throws IOException {
return fetch(sourceFileUrl, destinationFile, this.voldemortConfig.getHadoopConfigPath());
}
public File fetch(String sourceFileUrl, String destinationFile, String hadoopConfigPath)
throws IOException {
if(this.globalThrottleLimit != null) {
if(this.globalThrottleLimit.getSpeculativeRate() < this.minBytesPerSecond)
throw new VoldemortException("Too many push jobs.");
this.globalThrottleLimit.incrementNumJobs();
}
ObjectName jmxName = null;
try {
final Configuration config = new Configuration();
FileSystem fs = null;
config.setInt("io.socket.receive.buffer", bufferSize);
config.set("hadoop.rpc.socket.factory.class.ClientProtocol",
ConfigurableSocketFactory.class.getName());
config.set("hadoop.security.group.mapping",
"org.apache.hadoop.security.ShellBasedUnixGroupsMapping");
final Path path = new Path(sourceFileUrl);
if(hadoopConfigPath.length() > 0) {
config.addResource(new Path(hadoopConfigPath + "/core-site.xml"));
config.addResource(new Path(hadoopConfigPath + "/hdfs-site.xml"));
String security = config.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
if(security == null || !security.equals("kerberos")) {
logger.info("Security isn't turned on in the conf: "
+ CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION
+ " = "
+ config.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION));
logger.info("Fix that. Exiting.");
return null;
} else {
logger.info("Security is turned on in the conf. Trying to authenticate ...");
}
}
try {
if(HdfsFetcher.keytabPath.length() > 0) {
// First login using the specified principal and keytab file
UserGroupInformation.setConfiguration(config);
UserGroupInformation.loginUserFromKeytab(HdfsFetcher.kerberosPrincipal,
HdfsFetcher.keytabPath);
/*
* If login is successful, get the filesystem object. NOTE:
* Ideally we do not need a doAs block for this. Consider
* removing it in the future once the Hadoop jars have the
* corresponding patch (tracked in the Hadoop Apache
* project: HDFS-3367)
*/
try {
logger.info("I've logged in and am now Doasing as "
+ UserGroupInformation.getCurrentUser().getUserName());
fs = UserGroupInformation.getCurrentUser()
.doAs(new PrivilegedExceptionAction<FileSystem>() {
public FileSystem run() throws Exception {
FileSystem fs = path.getFileSystem(config);
return fs;
}
});
} catch(InterruptedException e) {
logger.error(e.getMessage());
} catch(Exception e) {
logger.error("Got an exception while getting the filesystem object: ");
logger.error("Exception class : " + e.getClass());
e.printStackTrace();
for(StackTraceElement et: e.getStackTrace()) {
logger.error(et.toString());
}
}
} else {
fs = path.getFileSystem(config);
}
} catch(IOException e) {
e.printStackTrace();
logger.error("Error in authenticating or getting the Filesystem object !!! Exiting !!!");
System.exit(-1);
}
CopyStats stats = new CopyStats(sourceFileUrl, sizeOfPath(fs, path));
jmxName = JmxUtils.registerMbean("hdfs-copy-" + copyCount.getAndIncrement(), stats);
File destination = new File(destinationFile);
if(destination.exists()) {
throw new VoldemortException("Version directory " + destination.getAbsolutePath()
+ " already exists");
}
logger.info("Starting fetch for : " + sourceFileUrl);
boolean result = fetch(fs, path, destination, stats);
logger.info("Completed fetch : " + sourceFileUrl);
// Close the filesystem
fs.close();
if(result) {
return destination;
} else {
return null;
}
} catch(Exception e) {
logger.error("Error while getting Hadoop filesystem : " + e);
return null;
} finally {
if(this.globalThrottleLimit != null) {
this.globalThrottleLimit.decrementNumJobs();
}
if(jmxName != null)
JmxUtils.unregisterMbean(jmxName);
}
}
private boolean fetch(FileSystem fs, Path source, File dest, CopyStats stats)
throws IOException {
if(!fs.isFile(source)) {
Utils.mkdirs(dest);
FileStatus[] statuses = fs.listStatus(source);
if(statuses != null) {
// sort the files so that index files come last. Maybe
// this will help keep them cached until the swap
Arrays.sort(statuses, new IndexFileLastComparator());
byte[] origCheckSum = null;
CheckSumType checkSumType = CheckSumType.NONE;
// Do a checksum of checksum - Similar to HDFS
CheckSum checkSumGenerator = null;
CheckSum fileCheckSumGenerator = null;
for(FileStatus status: statuses) {
// Kept for backwards compatibility
if(status.getPath().getName().contains("checkSum.txt")) {
// Ignore old checksum files
} else if(status.getPath().getName().contains(".metadata")) {
logger.debug("Reading .metadata");
// Read metadata into local file
File copyLocation = new File(dest, status.getPath().getName());
copyFileWithCheckSum(fs, status.getPath(), copyLocation, stats, null);
// Open the local file to initialize checksum
ReadOnlyStorageMetadata metadata;
try {
metadata = new ReadOnlyStorageMetadata(copyLocation);
} catch(IOException e) {
logger.error("Error reading metadata file ", e);
throw new VoldemortException(e);
}
// Read checksum
String checkSumTypeString = (String) metadata.get(ReadOnlyStorageMetadata.CHECKSUM_TYPE);
String checkSumString = (String) metadata.get(ReadOnlyStorageMetadata.CHECKSUM);
if(checkSumTypeString != null && checkSumString != null) {
try {
origCheckSum = Hex.decodeHex(checkSumString.toCharArray());
} catch(DecoderException e) {
logger.error("Exception reading checksum file. Ignoring checksum ",
e);
continue;
}
logger.debug("Checksum from .metadata "
+ new String(Hex.encodeHex(origCheckSum)));
checkSumType = CheckSum.fromString(checkSumTypeString);
checkSumGenerator = CheckSum.getInstance(checkSumType);
fileCheckSumGenerator = CheckSum.getInstance(checkSumType);
}
} else if(!status.getPath().getName().startsWith(".")) {
// Read other (.data , .index files)
File copyLocation = new File(dest, status.getPath().getName());
copyFileWithCheckSum(fs,
status.getPath(),
copyLocation,
stats,
fileCheckSumGenerator);
if(fileCheckSumGenerator != null && checkSumGenerator != null) {
byte[] checkSum = fileCheckSumGenerator.getCheckSum();
logger.debug("Checksum for " + status.getPath() + " - "
+ new String(Hex.encodeHex(checkSum)));
checkSumGenerator.update(checkSum);
}
}
}
logger.info("Completed reading all files from " + source.toString() + " to "
+ dest.getAbsolutePath());
// Check checksum
if(checkSumType != CheckSumType.NONE) {
byte[] newCheckSum = checkSumGenerator.getCheckSum();
boolean checkSumComparison = (ByteUtils.compare(newCheckSum, origCheckSum) == 0);
logger.info("Checksum generated from streaming - "
+ new String(Hex.encodeHex(newCheckSum)));
logger.info("Checksum on file - " + new String(Hex.encodeHex(origCheckSum)));
logger.info("Check-sum verification - " + checkSumComparison);
return checkSumComparison;
} else {
logger.info("No check-sum verification required");
return true;
}
}
}
logger.error("Source " + source.toString() + " should be a directory");
return false;
}
private void copyFileWithCheckSum(FileSystem fs,
Path source,
File dest,
CopyStats stats,
CheckSum fileCheckSumGenerator) throws IOException {
logger.debug("Starting copy of " + source + " to " + dest);
FSDataInputStream input = null;
OutputStream output = null;
for(int attempt = 0; attempt < NUM_RETRIES; attempt++) {
boolean success = true;
try {
input = fs.open(source);
output = new BufferedOutputStream(new FileOutputStream(dest));
byte[] buffer = new byte[bufferSize];
while(true) {
int read = input.read(buffer);
if(read < 0) {
break;
} else {
output.write(buffer, 0, read);
}
if(fileCheckSumGenerator != null)
fileCheckSumGenerator.update(buffer, 0, read);
if(throttler != null)
throttler.maybeThrottle(read);
stats.recordBytes(read);
if(stats.getBytesSinceLastReport() > reportingIntervalBytes) {
NumberFormat format = NumberFormat.getNumberInstance();
format.setMaximumFractionDigits(2);
logger.info(stats.getTotalBytesCopied() / (1024 * 1024) + " MB copied at "
+ format.format(stats.getBytesPerSecond() / (1024 * 1024))
+ " MB/sec - " + format.format(stats.getPercentCopied())
+ " % complete, destination:" + dest);
if(this.status != null) {
this.status.setStatus(stats.getTotalBytesCopied()
/ (1024 * 1024)
+ " MB copied at "
+ format.format(stats.getBytesPerSecond()
/ (1024 * 1024)) + " MB/sec - "
+ format.format(stats.getPercentCopied())
+ " % complete, destination:" + dest);
}
stats.reset();
}
}
logger.info("Completed copy of " + source + " to " + dest);
} catch(IOException ioe) {
success = false;
logger.error("Error during copying file ", ioe);
ioe.printStackTrace();
if(attempt < NUM_RETRIES - 1) {
logger.info("retrying copying");
} else {
throw ioe;
}
} finally {
IOUtils.closeQuietly(output);
IOUtils.closeQuietly(input);
if(success) {
break;
}
}
logger.debug("Completed copy of " + source + " to " + dest);
}
}
private long sizeOfPath(FileSystem fs, Path path) throws IOException {
long size = 0;
FileStatus[] statuses = fs.listStatus(path);
if(statuses != null) {
for(FileStatus status: statuses) {
if(status.isDir())
size += sizeOfPath(fs, status.getPath());
else
size += status.getLen();
}
}
return size;
}
public static class CopyStats {
private final String fileName;
private volatile long bytesSinceLastReport;
private volatile long totalBytesCopied;
private volatile long lastReportNs;
private volatile long totalBytes;
public CopyStats(String fileName, long totalBytes) {
this.fileName = fileName;
this.totalBytesCopied = 0L;
this.bytesSinceLastReport = 0L;
this.totalBytes = totalBytes;
this.lastReportNs = System.nanoTime();
}
public void recordBytes(long bytes) {
this.totalBytesCopied += bytes;
this.bytesSinceLastReport += bytes;
}
public void reset() {
this.bytesSinceLastReport = 0;
this.lastReportNs = System.nanoTime();
}
public long getBytesSinceLastReport() {
return bytesSinceLastReport;
}
public double getPercentCopied() {
if(totalBytes == 0) {
return 0.0;
} else {
return (double) (totalBytesCopied * 100) / (double) totalBytes;
}
}
@JmxGetter(name = "totalBytesCopied", description = "The total number of bytes copied so far in this transfer.")
public long getTotalBytesCopied() {
return totalBytesCopied;
}
@JmxGetter(name = "bytesPerSecond", description = "The rate of the transfer in bytes/second.")
public double getBytesPerSecond() {
double ellapsedSecs = (System.nanoTime() - lastReportNs) / (double) Time.NS_PER_SECOND;
return bytesSinceLastReport / ellapsedSecs;
}
@JmxGetter(name = "filename", description = "The file path being copied.")
public String getFilename() {
return this.fileName;
}
}
/**
* A comparator that sorts index files last. This is a heuristic for
* retaining the index file in page cache until the swap occurs
*
*/
public static class IndexFileLastComparator implements Comparator<FileStatus> {
public int compare(FileStatus fs1, FileStatus fs2) {
// directories before files
if(fs1.isDir())
return fs2.isDir() ? 0 : -1;
if(fs2.isDir())
return fs1.isDir() ? 0 : 1;
String f1 = fs1.getPath().getName(), f2 = fs2.getPath().getName();
// All metadata files given priority
if(f1.endsWith("metadata"))
return -1;
if(f2.endsWith("metadata"))
return 1;
// if both same, lexicographically
if((f1.endsWith(".index") && f2.endsWith(".index"))
|| (f1.endsWith(".data") && f2.endsWith(".data"))) {
return f1.compareToIgnoreCase(f2);
}
if(f1.endsWith(".index")) {
return 1;
} else {
return -1;
}
}
}
public void setAsyncOperationStatus(AsyncOperationStatus status) {
this.status = status;
}
/*
* Main method for testing fetching
*/
public static void main(String[] args) throws Exception {
if(args.length < 1)
Utils.croak("USAGE: java " + HdfsFetcher.class.getName()
+ " url [keytab location] [kerberos username] [hadoop-config-path]");
String url = args[0];
String keytabLocation = "";
String kerberosUser = "";
String hadoopPath = "";
if(args.length == 4) {
keytabLocation = args[1];
kerberosUser = args[2];
hadoopPath = args[3];
}
long maxBytesPerSec = 1024 * 1024 * 1024;
Path p = new Path(url);
final Configuration config = new Configuration();
final URI uri = new URI(url);
config.setInt("io.file.buffer.size", VoldemortConfig.DEFAULT_BUFFER_SIZE);
config.set("hadoop.rpc.socket.factory.class.ClientProtocol",
ConfigurableSocketFactory.class.getName());
config.setInt("io.socket.receive.buffer", 1 * 1024 * 1024 - 10000);
FileSystem fs = null;
p = new Path(url);
HdfsFetcher.keytabPath = keytabLocation;
HdfsFetcher.kerberosPrincipal = kerberosUser;
if(hadoopPath.length() > 0) {
config.set("hadoop.security.group.mapping",
"org.apache.hadoop.security.ShellBasedUnixGroupsMapping");
config.addResource(new Path(hadoopPath + "/core-site.xml"));
config.addResource(new Path(hadoopPath + "/hdfs-site.xml"));
String security = config.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
if(security == null || !security.equals("kerberos")) {
logger.info("Security isn't turned on in the conf: "
+ CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION + " = "
+ config.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION));
logger.info("Fix that. Exiting.");
return;
} else {
logger.info("Security is turned on in the conf. Trying to authenticate ...");
}
}
try {
// Get the filesystem object
if(keytabLocation.length() > 0) {
UserGroupInformation.setConfiguration(config);
UserGroupInformation.loginUserFromKeytab(kerberosUser, keytabLocation);
final Path path = p;
try {
logger.debug("I've logged in and am now Doasing as "
+ UserGroupInformation.getCurrentUser().getUserName());
fs = UserGroupInformation.getCurrentUser()
.doAs(new PrivilegedExceptionAction<FileSystem>() {
public FileSystem run() throws Exception {
FileSystem fs = path.getFileSystem(config);
return fs;
}
});
} catch(InterruptedException e) {
logger.error(e.getMessage());
} catch(Exception e) {
logger.error("Got an exception while getting the filesystem object: ");
logger.error("Exception class : " + e.getClass());
e.printStackTrace();
for(StackTraceElement et: e.getStackTrace()) {
logger.error(et.toString());
}
}
} else {
fs = p.getFileSystem(config);
}
} catch(IOException e) {
e.printStackTrace();
System.err.println("Error !!! Exiting !!!");
System.exit(-1);
}
FileStatus status = fs.listStatus(p)[0];
long size = status.getLen();
HdfsFetcher fetcher = new HdfsFetcher(null,
maxBytesPerSec,
VoldemortConfig.REPORTING_INTERVAL_BYTES,
VoldemortConfig.DEFAULT_BUFFER_SIZE,
0,
keytabLocation,
kerberosUser);
long start = System.currentTimeMillis();
File location = fetcher.fetch(url, System.getProperty("java.io.tmpdir") + File.separator
+ start, hadoopPath);
double rate = size * Time.MS_PER_SECOND / (double) (System.currentTimeMillis() - start);
NumberFormat nf = NumberFormat.getInstance();
nf.setMaximumFractionDigits(2);
System.out.println("Fetch to " + location + " completed: "
+ nf.format(rate / (1024.0 * 1024.0)) + " MB/sec.");
fs.close();
}
}