/
BlockReceiver.java
1459 lines (1341 loc) · 54 KB
/
BlockReceiver.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
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
import java.io.BufferedOutputStream;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileDescriptor;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.io.Writer;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.zip.Checksum;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
/** A class that receives a block and writes to its own disk, meanwhile
* may copies it to another site. If a throttler is provided,
* streaming throttling is also supported.
**/
class BlockReceiver implements Closeable {
public static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
@VisibleForTesting
static long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024;
private final long datanodeSlowLogThresholdMs;
private DataInputStream in = null; // from where data are read
private DataChecksum clientChecksum; // checksum used by client
private DataChecksum diskChecksum; // checksum we write to disk
/**
* In the case that the client is writing with a different
* checksum polynomial than the block is stored with on disk,
* the DataNode needs to recalculate checksums before writing.
*/
private final boolean needsChecksumTranslation;
private OutputStream out = null; // to block file at local disk
private FileDescriptor outFd;
private DataOutputStream checksumOut = null; // to crc file at local disk
private final int bytesPerChecksum;
private final int checksumSize;
private final PacketReceiver packetReceiver = new PacketReceiver(false);
protected final String inAddr;
protected final String myAddr;
private String mirrorAddr;
private DataOutputStream mirrorOut;
private Daemon responder = null;
private DataTransferThrottler throttler;
private ReplicaOutputStreams streams;
private DatanodeInfo srcDataNode = null;
private final DataNode datanode;
volatile private boolean mirrorError;
// Cache management state
private boolean dropCacheBehindWrites;
private long lastCacheManagementOffset = 0;
private boolean syncBehindWrites;
private boolean syncBehindWritesInBackground;
/** The client name. It is empty if a datanode is the client */
private final String clientname;
private final boolean isClient;
private final boolean isDatanode;
/** the block to receive */
private final ExtendedBlock block;
/** the replica to write */
private final ReplicaInPipelineInterface replicaInfo;
/** pipeline stage */
private final BlockConstructionStage stage;
private final boolean isTransfer;
private boolean syncOnClose;
private long restartBudget;
/** the reference of the volume where the block receiver writes to */
private ReplicaHandler replicaHandler;
/**
* for replaceBlock response
*/
private final long responseInterval;
private long lastResponseTime = 0;
private boolean isReplaceBlock = false;
private DataOutputStream replyOut = null;
private boolean pinning;
BlockReceiver(final ExtendedBlock block, final StorageType storageType,
final DataInputStream in,
final String inAddr, final String myAddr,
final BlockConstructionStage stage,
final long newGs, final long minBytesRcvd, final long maxBytesRcvd,
final String clientname, final DatanodeInfo srcDataNode,
final DataNode datanode, DataChecksum requestedChecksum,
CachingStrategy cachingStrategy,
final boolean allowLazyPersist,
final boolean pinning) throws IOException {
try{
this.block = block;
this.in = in;
this.inAddr = inAddr;
this.myAddr = myAddr;
this.srcDataNode = srcDataNode;
this.datanode = datanode;
this.clientname = clientname;
this.isDatanode = clientname.length() == 0;
this.isClient = !this.isDatanode;
this.restartBudget = datanode.getDnConf().restartReplicaExpiry;
this.datanodeSlowLogThresholdMs = datanode.getDnConf().datanodeSlowIoWarningThresholdMs;
// For replaceBlock() calls response should be sent to avoid socketTimeout
// at clients. So sending with the interval of 0.5 * socketTimeout
this.responseInterval = (long) (datanode.getDnConf().socketTimeout * 0.5);
//for datanode, we have
//1: clientName.length() == 0, and
//2: stage == null or PIPELINE_SETUP_CREATE
this.stage = stage;
this.isTransfer = stage == BlockConstructionStage.TRANSFER_RBW
|| stage == BlockConstructionStage.TRANSFER_FINALIZED;
this.pinning = pinning;
if (LOG.isDebugEnabled()) {
LOG.debug(getClass().getSimpleName() + ": " + block
+ "\n isClient =" + isClient + ", clientname=" + clientname
+ "\n isDatanode=" + isDatanode + ", srcDataNode=" + srcDataNode
+ "\n inAddr=" + inAddr + ", myAddr=" + myAddr
+ "\n cachingStrategy = " + cachingStrategy
+ "\n pinning=" + pinning
);
}
//
// Open local disk out
//
if (isDatanode) { //replication or move
replicaHandler = datanode.data.createTemporary(storageType, block);
} else {
switch (stage) {
case PIPELINE_SETUP_CREATE:
replicaHandler = datanode.data.createRbw(storageType, block, allowLazyPersist);
datanode.notifyNamenodeReceivingBlock(
block, replicaHandler.getReplica().getStorageUuid());
break;
case PIPELINE_SETUP_STREAMING_RECOVERY:
replicaHandler = datanode.data.recoverRbw(
block, newGs, minBytesRcvd, maxBytesRcvd);
block.setGenerationStamp(newGs);
break;
case PIPELINE_SETUP_APPEND:
replicaHandler = datanode.data.append(block, newGs, minBytesRcvd);
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(
block, replicaHandler.getReplica().getStorageUuid());
break;
case PIPELINE_SETUP_APPEND_RECOVERY:
replicaHandler = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(
block, replicaHandler.getReplica().getStorageUuid());
break;
case TRANSFER_RBW:
case TRANSFER_FINALIZED:
// this is a transfer destination
replicaHandler =
datanode.data.createTemporary(storageType, block);
break;
default: throw new IOException("Unsupported stage " + stage +
" while receiving block " + block + " from " + inAddr);
}
}
replicaInfo = replicaHandler.getReplica();
this.dropCacheBehindWrites = (cachingStrategy.getDropBehind() == null) ?
datanode.getDnConf().dropCacheBehindWrites :
cachingStrategy.getDropBehind();
this.syncBehindWrites = datanode.getDnConf().syncBehindWrites;
this.syncBehindWritesInBackground = datanode.getDnConf().
syncBehindWritesInBackground;
final boolean isCreate = isDatanode || isTransfer
|| stage == BlockConstructionStage.PIPELINE_SETUP_CREATE;
streams = replicaInfo.createStreams(isCreate, requestedChecksum);
assert streams != null : "null streams!";
// read checksum meta information
this.clientChecksum = requestedChecksum;
this.diskChecksum = streams.getChecksum();
this.needsChecksumTranslation = !clientChecksum.equals(diskChecksum);
this.bytesPerChecksum = diskChecksum.getBytesPerChecksum();
this.checksumSize = diskChecksum.getChecksumSize();
this.out = streams.getDataOut();
if (out instanceof FileOutputStream) {
this.outFd = ((FileOutputStream)out).getFD();
} else {
LOG.warn("Could not get file descriptor for outputstream of class " +
out.getClass());
}
this.checksumOut = new DataOutputStream(new BufferedOutputStream(
streams.getChecksumOut(), HdfsConstants.SMALL_BUFFER_SIZE));
// write data chunk header if creating a new replica
if (isCreate) {
BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);
}
} catch (ReplicaAlreadyExistsException bae) {
throw bae;
} catch (ReplicaNotFoundException bne) {
throw bne;
} catch(IOException ioe) {
IOUtils.closeStream(this);
cleanupBlock();
// check if there is a disk error
IOException cause = DatanodeUtil.getCauseIfDiskError(ioe);
DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ",
cause);
if (cause != null) { // possible disk error
ioe = cause;
datanode.checkDiskErrorAsync();
}
throw ioe;
}
}
/** Return the datanode object. */
DataNode getDataNode() {return datanode;}
String getStorageUuid() {
return replicaInfo.getStorageUuid();
}
/**
* close files.
*/
@Override
public void close() throws IOException {
packetReceiver.close();
IOException ioe = null;
if (syncOnClose && (out != null || checksumOut != null)) {
datanode.metrics.incrFsyncCount();
}
long flushTotalNanos = 0;
boolean measuredFlushTime = false;
// close checksum file
try {
if (checksumOut != null) {
long flushStartNanos = System.nanoTime();
checksumOut.flush();
long flushEndNanos = System.nanoTime();
if (syncOnClose) {
long fsyncStartNanos = flushEndNanos;
streams.syncChecksumOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
}
flushTotalNanos += flushEndNanos - flushStartNanos;
measuredFlushTime = true;
checksumOut.close();
checksumOut = null;
}
} catch(IOException e) {
ioe = e;
}
finally {
IOUtils.closeStream(checksumOut);
}
// close block file
try {
if (out != null) {
long flushStartNanos = System.nanoTime();
out.flush();
long flushEndNanos = System.nanoTime();
if (syncOnClose) {
long fsyncStartNanos = flushEndNanos;
streams.syncDataOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
}
flushTotalNanos += flushEndNanos - flushStartNanos;
measuredFlushTime = true;
out.close();
out = null;
}
} catch (IOException e) {
ioe = e;
}
finally{
IOUtils.closeStream(out);
}
if (replicaHandler != null) {
IOUtils.cleanup(null, replicaHandler);
replicaHandler = null;
}
if (measuredFlushTime) {
datanode.metrics.addFlushNanos(flushTotalNanos);
}
// disk check
if(ioe != null) {
datanode.checkDiskErrorAsync();
throw ioe;
}
}
/**
* Flush block data and metadata files to disk.
* @throws IOException
*/
void flushOrSync(boolean isSync) throws IOException {
long flushTotalNanos = 0;
long begin = Time.monotonicNow();
if (checksumOut != null) {
long flushStartNanos = System.nanoTime();
checksumOut.flush();
long flushEndNanos = System.nanoTime();
if (isSync) {
long fsyncStartNanos = flushEndNanos;
streams.syncChecksumOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
}
flushTotalNanos += flushEndNanos - flushStartNanos;
}
if (out != null) {
long flushStartNanos = System.nanoTime();
out.flush();
long flushEndNanos = System.nanoTime();
if (isSync) {
long fsyncStartNanos = flushEndNanos;
streams.syncDataOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
}
flushTotalNanos += flushEndNanos - flushStartNanos;
}
if (checksumOut != null || out != null) {
datanode.metrics.addFlushNanos(flushTotalNanos);
if (isSync) {
datanode.metrics.incrFsyncCount();
}
}
long duration = Time.monotonicNow() - begin;
if (duration > datanodeSlowLogThresholdMs) {
LOG.warn("Slow flushOrSync took " + duration + "ms (threshold="
+ datanodeSlowLogThresholdMs + "ms), isSync:" + isSync + ", flushTotalNanos="
+ flushTotalNanos + "ns");
}
}
/**
* While writing to mirrorOut, failure to write to mirror should not
* affect this datanode unless it is caused by interruption.
*/
private void handleMirrorOutError(IOException ioe) throws IOException {
String bpid = block.getBlockPoolId();
LOG.info(datanode.getDNRegistrationForBP(bpid)
+ ":Exception writing " + block + " to mirror " + mirrorAddr, ioe);
if (Thread.interrupted()) { // shut down if the thread is interrupted
throw ioe;
} else { // encounter an error while writing to mirror
// continue to run even if can not write to mirror
// notify client of the error
// and wait for the client to shut down the pipeline
mirrorError = true;
}
}
/**
* Verify multiple CRC chunks.
*/
private void verifyChunks(ByteBuffer dataBuf, ByteBuffer checksumBuf)
throws IOException {
try {
clientChecksum.verifyChunkedSums(dataBuf, checksumBuf, clientname, 0);
} catch (ChecksumException ce) {
LOG.warn("Checksum error in block " + block + " from " + inAddr, ce);
// No need to report to namenode when client is writing.
if (srcDataNode != null && isDatanode) {
try {
LOG.info("report corrupt " + block + " from datanode " +
srcDataNode + " to namenode");
datanode.reportRemoteBadBlock(srcDataNode, block);
} catch (IOException e) {
LOG.warn("Failed to report bad " + block +
" from datanode " + srcDataNode + " to namenode");
}
}
throw new IOException("Unexpected checksum mismatch while writing "
+ block + " from " + inAddr);
}
}
/**
* Translate CRC chunks from the client's checksum implementation
* to the disk checksum implementation.
*
* This does not verify the original checksums, under the assumption
* that they have already been validated.
*/
private void translateChunks(ByteBuffer dataBuf, ByteBuffer checksumBuf) {
diskChecksum.calculateChunkedSums(dataBuf, checksumBuf);
}
/**
* Check whether checksum needs to be verified.
* Skip verifying checksum iff this is not the last one in the
* pipeline and clientName is non-null. i.e. Checksum is verified
* on all the datanodes when the data is being written by a
* datanode rather than a client. Whe client is writing the data,
* protocol includes acks and only the last datanode needs to verify
* checksum.
* @return true if checksum verification is needed, otherwise false.
*/
private boolean shouldVerifyChecksum() {
return (mirrorOut == null || isDatanode || needsChecksumTranslation);
}
/**
* Receives and processes a packet. It can contain many chunks.
* returns the number of data bytes that the packet has.
*/
private int receivePacket() throws IOException {
// read the next packet
packetReceiver.receiveNextPacket(in);
PacketHeader header = packetReceiver.getHeader();
if (LOG.isDebugEnabled()){
LOG.debug("Receiving one packet for block " + block +
": " + header);
}
// Sanity check the header
if (header.getOffsetInBlock() > replicaInfo.getNumBytes()) {
throw new IOException("Received an out-of-sequence packet for " + block +
"from " + inAddr + " at offset " + header.getOffsetInBlock() +
". Expecting packet starting at " + replicaInfo.getNumBytes());
}
if (header.getDataLen() < 0) {
throw new IOException("Got wrong length during writeBlock(" + block +
") from " + inAddr + " at offset " +
header.getOffsetInBlock() + ": " +
header.getDataLen());
}
long offsetInBlock = header.getOffsetInBlock();
long seqno = header.getSeqno();
boolean lastPacketInBlock = header.isLastPacketInBlock();
final int len = header.getDataLen();
boolean syncBlock = header.getSyncBlock();
// avoid double sync'ing on close
if (syncBlock && lastPacketInBlock) {
this.syncOnClose = false;
}
// update received bytes
final long firstByteInBlock = offsetInBlock;
offsetInBlock += len;
if (replicaInfo.getNumBytes() < offsetInBlock) {
replicaInfo.setNumBytes(offsetInBlock);
}
// put in queue for pending acks, unless sync was requested
if (responder != null && !syncBlock && !shouldVerifyChecksum()) {
((PacketResponder) responder.getRunnable()).enqueue(seqno,
lastPacketInBlock, offsetInBlock, Status.SUCCESS);
}
//First write the packet to the mirror:
if (mirrorOut != null && !mirrorError) {
try {
long begin = Time.monotonicNow();
packetReceiver.mirrorPacketTo(mirrorOut);
mirrorOut.flush();
long duration = Time.monotonicNow() - begin;
if (duration > datanodeSlowLogThresholdMs) {
LOG.warn("Slow BlockReceiver write packet to mirror took " + duration
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
}
} catch (IOException e) {
handleMirrorOutError(e);
}
}
ByteBuffer dataBuf = packetReceiver.getDataSlice();
ByteBuffer checksumBuf = packetReceiver.getChecksumSlice();
if (lastPacketInBlock || len == 0) {
if(LOG.isDebugEnabled()) {
LOG.debug("Receiving an empty packet or the end of the block " + block);
}
// sync block if requested
if (syncBlock) {
flushOrSync(true);
}
} else {
final int checksumLen = diskChecksum.getChecksumSize(len);
final int checksumReceivedLen = checksumBuf.capacity();
if (checksumReceivedLen > 0 && checksumReceivedLen != checksumLen) {
throw new IOException("Invalid checksum length: received length is "
+ checksumReceivedLen + " but expected length is " + checksumLen);
}
if (checksumReceivedLen > 0 && shouldVerifyChecksum()) {
try {
verifyChunks(dataBuf, checksumBuf);
} catch (IOException ioe) {
// checksum error detected locally. there is no reason to continue.
if (responder != null) {
try {
((PacketResponder) responder.getRunnable()).enqueue(seqno,
lastPacketInBlock, offsetInBlock,
Status.ERROR_CHECKSUM);
// Wait until the responder sends back the response
// and interrupt this thread.
Thread.sleep(3000);
} catch (InterruptedException e) { }
}
throw new IOException("Terminating due to a checksum error." + ioe);
}
if (needsChecksumTranslation) {
// overwrite the checksums in the packet buffer with the
// appropriate polynomial for the disk storage.
translateChunks(dataBuf, checksumBuf);
}
}
if (checksumReceivedLen == 0 && !streams.isTransientStorage()) {
// checksum is missing, need to calculate it
checksumBuf = ByteBuffer.allocate(checksumLen);
diskChecksum.calculateChunkedSums(dataBuf, checksumBuf);
}
// by this point, the data in the buffer uses the disk checksum
final boolean shouldNotWriteChecksum = checksumReceivedLen == 0
&& streams.isTransientStorage();
try {
long onDiskLen = replicaInfo.getBytesOnDisk();
if (onDiskLen<offsetInBlock) {
//finally write to the disk :
if (onDiskLen % bytesPerChecksum != 0) {
// prepare to overwrite last checksum
adjustCrcFilePosition();
}
// If this is a partial chunk, then read in pre-existing checksum
Checksum partialCrc = null;
if (!shouldNotWriteChecksum && firstByteInBlock % bytesPerChecksum != 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("receivePacket for " + block
+ ": bytesPerChecksum=" + bytesPerChecksum
+ " does not divide firstByteInBlock=" + firstByteInBlock);
}
long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
onDiskLen / bytesPerChecksum * checksumSize;
partialCrc = computePartialChunkCrc(onDiskLen, offsetInChecksum);
}
int startByteToDisk = (int)(onDiskLen-firstByteInBlock)
+ dataBuf.arrayOffset() + dataBuf.position();
int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
// Write data to disk.
long begin = Time.monotonicNow();
out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
long duration = Time.monotonicNow() - begin;
if (duration > datanodeSlowLogThresholdMs) {
LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
}
final byte[] lastCrc;
if (shouldNotWriteChecksum) {
lastCrc = null;
} else if (partialCrc != null) {
// If this is a partial chunk, then verify that this is the only
// chunk in the packet. Calculate new crc for this chunk.
if (len > bytesPerChecksum) {
throw new IOException("Unexpected packet data length for "
+ block + " from " + inAddr + ": a partial chunk must be "
+ " sent in an individual packet (data length = " + len
+ " > bytesPerChecksum = " + bytesPerChecksum + ")");
}
partialCrc.update(dataBuf.array(), startByteToDisk, numBytesToDisk);
byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
lastCrc = copyLastChunkChecksum(buf, checksumSize, buf.length);
checksumOut.write(buf);
if(LOG.isDebugEnabled()) {
LOG.debug("Writing out partial crc for data len " + len);
}
partialCrc = null;
} else {
// write checksum
final int offset = checksumBuf.arrayOffset() +
checksumBuf.position();
final int end = offset + checksumLen;
lastCrc = copyLastChunkChecksum(checksumBuf.array(), checksumSize,
end);
checksumOut.write(checksumBuf.array(), offset, checksumLen);
}
/// flush entire packet, sync if requested
flushOrSync(syncBlock);
replicaInfo.setLastChecksumAndDataLen(offsetInBlock, lastCrc);
datanode.metrics.incrBytesWritten(len);
datanode.metrics.incrTotalWriteTime(duration);
manageWriterOsCache(offsetInBlock);
}
} catch (IOException iex) {
datanode.checkDiskErrorAsync();
throw iex;
}
}
// if sync was requested, put in queue for pending acks here
// (after the fsync finished)
if (responder != null && (syncBlock || shouldVerifyChecksum())) {
((PacketResponder) responder.getRunnable()).enqueue(seqno,
lastPacketInBlock, offsetInBlock, Status.SUCCESS);
}
/*
* Send in-progress responses for the replaceBlock() calls back to caller to
* avoid timeouts due to balancer throttling. HDFS-6247
*/
if (isReplaceBlock
&& (Time.monotonicNow() - lastResponseTime > responseInterval)) {
BlockOpResponseProto.Builder response = BlockOpResponseProto.newBuilder()
.setStatus(Status.IN_PROGRESS);
response.build().writeDelimitedTo(replyOut);
replyOut.flush();
lastResponseTime = Time.monotonicNow();
}
if (throttler != null) { // throttle I/O
throttler.throttle(len);
}
return lastPacketInBlock?-1:len;
}
private static byte[] copyLastChunkChecksum(byte[] array, int size, int end) {
return Arrays.copyOfRange(array, end - size, end);
}
private void manageWriterOsCache(long offsetInBlock) {
try {
if (outFd != null &&
offsetInBlock > lastCacheManagementOffset + CACHE_DROP_LAG_BYTES) {
long begin = Time.monotonicNow();
//
// For SYNC_FILE_RANGE_WRITE, we want to sync from
// lastCacheManagementOffset to a position "two windows ago"
//
// <========= sync ===========>
// +-----------------------O--------------------------X
// start last curPos
// of file
//
if (syncBehindWrites) {
if (syncBehindWritesInBackground) {
this.datanode.getFSDataset().submitBackgroundSyncFileRangeRequest(
block, outFd, lastCacheManagementOffset,
offsetInBlock - lastCacheManagementOffset,
NativeIO.POSIX.SYNC_FILE_RANGE_WRITE);
} else {
NativeIO.POSIX.syncFileRangeIfPossible(outFd,
lastCacheManagementOffset, offsetInBlock
- lastCacheManagementOffset,
NativeIO.POSIX.SYNC_FILE_RANGE_WRITE);
}
}
//
// For POSIX_FADV_DONTNEED, we want to drop from the beginning
// of the file to a position prior to the current position.
//
// <=== drop =====>
// <---W--->
// +--------------+--------O--------------------------X
// start dropPos last curPos
// of file
//
long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
if (dropPos > 0 && dropCacheBehindWrites) {
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
block.getBlockName(), outFd, 0, dropPos,
NativeIO.POSIX.POSIX_FADV_DONTNEED);
}
lastCacheManagementOffset = offsetInBlock;
long duration = Time.monotonicNow() - begin;
if (duration > datanodeSlowLogThresholdMs) {
LOG.warn("Slow manageWriterOsCache took " + duration
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
}
}
} catch (Throwable t) {
LOG.warn("Error managing cache for writer of block " + block, t);
}
}
public void sendOOB() throws IOException, InterruptedException {
((PacketResponder) responder.getRunnable()).sendOOBResponse(PipelineAck
.getRestartOOBStatus());
}
void receiveBlock(
DataOutputStream mirrOut, // output to next datanode
DataInputStream mirrIn, // input from next datanode
DataOutputStream replyOut, // output to previous datanode
String mirrAddr, DataTransferThrottler throttlerArg,
DatanodeInfo[] downstreams,
boolean isReplaceBlock) throws IOException {
syncOnClose = datanode.getDnConf().syncOnClose;
boolean responderClosed = false;
mirrorOut = mirrOut;
mirrorAddr = mirrAddr;
throttler = throttlerArg;
this.replyOut = replyOut;
this.isReplaceBlock = isReplaceBlock;
try {
if (isClient && !isTransfer) {
responder = new Daemon(datanode.threadGroup,
new PacketResponder(replyOut, mirrIn, downstreams));
responder.start(); // start thread to processes responses
}
while (receivePacket() >= 0) { /* Receive until the last packet */ }
// wait for all outstanding packet responses. And then
// indicate responder to gracefully shutdown.
// Mark that responder has been closed for future processing
if (responder != null) {
((PacketResponder)responder.getRunnable()).close();
responderClosed = true;
}
// If this write is for a replication or transfer-RBW/Finalized,
// then finalize block or convert temporary to RBW.
// For client-writes, the block is finalized in the PacketResponder.
if (isDatanode || isTransfer) {
// close the block/crc files
close();
block.setNumBytes(replicaInfo.getNumBytes());
if (stage == BlockConstructionStage.TRANSFER_RBW) {
// for TRANSFER_RBW, convert temporary to RBW
datanode.data.convertTemporaryToRbw(block);
} else {
// for isDatnode or TRANSFER_FINALIZED
// Finalize the block.
datanode.data.finalizeBlock(block);
}
datanode.metrics.incrBlocksWritten();
}
} catch (IOException ioe) {
if (datanode.isRestarting()) {
// Do not throw if shutting down for restart. Otherwise, it will cause
// premature termination of responder.
LOG.info("Shutting down for restart (" + block + ").");
} else {
LOG.info("Exception for " + block, ioe);
throw ioe;
}
} finally {
// Clear the previous interrupt state of this thread.
Thread.interrupted();
// If a shutdown for restart was initiated, upstream needs to be notified.
// There is no need to do anything special if the responder was closed
// normally.
if (!responderClosed) { // Data transfer was not complete.
if (responder != null) {
// In case this datanode is shutting down for quick restart,
// send a special ack upstream.
if (datanode.isRestarting() && isClient && !isTransfer) {
File blockFile = ((ReplicaInPipeline)replicaInfo).getBlockFile();
File restartMeta = new File(blockFile.getParent() +
File.pathSeparator + "." + blockFile.getName() + ".restart");
if (restartMeta.exists() && !restartMeta.delete()) {
LOG.warn("Failed to delete restart meta file: " +
restartMeta.getPath());
}
try (Writer out = new OutputStreamWriter(
new FileOutputStream(restartMeta), "UTF-8")) {
// write out the current time.
out.write(Long.toString(Time.now() + restartBudget));
out.flush();
} catch (IOException ioe) {
// The worst case is not recovering this RBW replica.
// Client will fall back to regular pipeline recovery.
} finally {
IOUtils.cleanup(LOG, out);
}
try {
// Even if the connection is closed after the ack packet is
// flushed, the client can react to the connection closure
// first. Insert a delay to lower the chance of client
// missing the OOB ack.
Thread.sleep(1000);
} catch (InterruptedException ie) {
// It is already going down. Ignore this.
}
}
responder.interrupt();
}
IOUtils.closeStream(this);
cleanupBlock();
}
if (responder != null) {
try {
responder.interrupt();
// join() on the responder should timeout a bit earlier than the
// configured deadline. Otherwise, the join() on this thread will
// likely timeout as well.
long joinTimeout = datanode.getDnConf().getXceiverStopTimeout();
joinTimeout = joinTimeout > 1 ? joinTimeout*8/10 : joinTimeout;
responder.join(joinTimeout);
if (responder.isAlive()) {
String msg = "Join on responder thread " + responder
+ " timed out";
LOG.warn(msg + "\n" + StringUtils.getStackTrace(responder));
throw new IOException(msg);
}
} catch (InterruptedException e) {
responder.interrupt();
// do not throw if shutting down for restart.
if (!datanode.isRestarting()) {
throw new IOException("Interrupted receiveBlock");
}
}
responder = null;
}
}
}
/** Cleanup a partial block
* if this write is for a replication request (and not from a client)
*/
private void cleanupBlock() throws IOException {
if (isDatanode) {
datanode.data.unfinalizeBlock(block);
}
}
/**
* Adjust the file pointer in the local meta file so that the last checksum
* will be overwritten.
*/
private void adjustCrcFilePosition() throws IOException {
if (out != null) {
out.flush();
}
if (checksumOut != null) {
checksumOut.flush();
}
// rollback the position of the meta file
datanode.data.adjustCrcChannelPosition(block, streams, checksumSize);
}
/**
* Convert a checksum byte array to a long
*/
static private long checksum2long(byte[] checksum) {
long crc = 0L;
for(int i=0; i<checksum.length; i++) {
crc |= (0xffL&checksum[i])<<((checksum.length-i-1)*8);
}
return crc;
}
/**
* reads in the partial crc chunk and computes checksum
* of pre-existing data in partial chunk.
*/
private Checksum computePartialChunkCrc(long blkoff, long ckoff)
throws IOException {
// find offset of the beginning of partial chunk.
//
int sizePartialChunk = (int) (blkoff % bytesPerChecksum);
blkoff = blkoff - sizePartialChunk;
if (LOG.isDebugEnabled()) {
LOG.debug("computePartialChunkCrc for " + block
+ ": sizePartialChunk=" + sizePartialChunk
+ ", block offset=" + blkoff
+ ", metafile offset=" + ckoff);
}
// create an input stream from the block file
// and read in partial crc chunk into temporary buffer
//
byte[] buf = new byte[sizePartialChunk];
byte[] crcbuf = new byte[checksumSize];
try (ReplicaInputStreams instr =
datanode.data.getTmpInputStreams(block, blkoff, ckoff)) {
IOUtils.readFully(instr.getDataIn(), buf, 0, sizePartialChunk);
// open meta file and read in crc value computer earlier
IOUtils.readFully(instr.getChecksumIn(), crcbuf, 0, crcbuf.length);
}
// compute crc of partial chunk from data read in the block file.
final Checksum partialCrc = DataChecksum.newDataChecksum(
diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum());
partialCrc.update(buf, 0, sizePartialChunk);
if (LOG.isDebugEnabled()) {
LOG.debug("Read in partial CRC chunk from disk for " + block);
}
// paranoia! verify that the pre-computed crc matches what we
// recalculated just now
if (partialCrc.getValue() != checksum2long(crcbuf)) {
String msg = "Partial CRC " + partialCrc.getValue() +
" does not match value computed the " +
" last time file was closed " +
checksum2long(crcbuf);
throw new IOException(msg);
}
return partialCrc;
}
private static enum PacketResponderType {
NON_PIPELINE, LAST_IN_PIPELINE, HAS_DOWNSTREAM_IN_PIPELINE
}
/**
* Processes responses from downstream datanodes in the pipeline
* and sends back replies to the originator.
*/
class PacketResponder implements Runnable, Closeable {
/** queue for packets waiting for ack - synchronization using monitor lock */
private final LinkedList<Packet> ackQueue = new LinkedList<Packet>();
/** the thread that spawns this responder */
private final Thread receiverThread = Thread.currentThread();
/** is this responder running? - synchronization using monitor lock */
private volatile boolean running = true;
/** input from the next downstream datanode */
private final DataInputStream downstreamIn;