-
Notifications
You must be signed in to change notification settings - Fork 398
/
WALFile.java
1068 lines (938 loc) · 32.7 KB
/
WALFile.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
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package io.confluent.connect.hdfs.wal;
import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.VersionMismatchException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.io.serializer.Serializer;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.Options;
import org.apache.hadoop.util.Time;
import org.apache.kafka.connect.errors.ConnectException;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.IOException;
import java.rmi.server.UID;
import java.security.MessageDigest;
import java.util.Arrays;
import io.confluent.connect.hdfs.HdfsSinkConnectorConfig;
public class WALFile {
private static final Log log = LogFactory.getLog(WALFile.class);
private static final byte INITIAL_VERSION = (byte) 0;
private static final int SYNC_ESCAPE = -1; // "length" of sync entries
private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash
private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash
/**
* The number of bytes between sync points.
*/
public static final int SYNC_INTERVAL = 100 * SYNC_SIZE;
private static byte[] VERSION = new byte[]{
(byte) 'W', (byte) 'A', (byte) 'L', INITIAL_VERSION
};
private static String deserErrorFmt = "Could not find a deserializer for the %s class: '%s'. "
+ "Please ensure that the configuration '%s' is properly configured, if you're using custom"
+ " serialization.";
private WALFile() {}
public static Writer createWriter(
HdfsSinkConnectorConfig conf,
Writer.Option... opts
) throws IOException {
return new Writer(conf, opts);
}
/**
* Get the configured buffer size
*/
private static int getBufferSize(Configuration conf) {
return conf.getInt("io.file.buffer.size", 4096);
}
public static class Writer implements Closeable, Syncable {
protected Serializer<WALEntry> keySerializer;
protected Serializer<WALEntry> valSerializer;
boolean ownOutputStream = true;
// Insert a globally unique 16-byte value every few entries, so that one
// can seek into the middle of a file and then synchronize with record
// starts and ends by scanning for this value.
long lastSyncPos; // position of last sync
byte[] sync; // 16 random bytes
private FileSystem fs;
private FSDataOutputStream out;
private DataOutputBuffer buffer = new DataOutputBuffer();
private boolean appendMode;
{
try {
MessageDigest digester = MessageDigest.getInstance("MD5");
long time = Time.now();
digester.update((new UID() + "@" + time).getBytes(Charsets.UTF_8));
sync = digester.digest();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
Writer(HdfsSinkConnectorConfig connectorConfig, Option... opts) throws IOException {
Configuration conf = connectorConfig.getHadoopConfiguration();
BlockSizeOption blockSizeOption =
Options.getOption(BlockSizeOption.class, opts);
BufferSizeOption bufferSizeOption =
Options.getOption(BufferSizeOption.class, opts);
ReplicationOption replicationOption =
Options.getOption(ReplicationOption.class, opts);
FileOption fileOption = Options.getOption(FileOption.class, opts);
AppendIfExistsOption appendIfExistsOption = Options.getOption(
AppendIfExistsOption.class, opts);
StreamOption streamOption = Options.getOption(StreamOption.class, opts);
// check consistency of options
if ((fileOption == null) == (streamOption == null)) {
throw new IllegalArgumentException("file or stream must be specified");
}
if (fileOption == null && (
blockSizeOption != null
|| bufferSizeOption != null
|| replicationOption != null
)
) {
throw new IllegalArgumentException("file modifier options not compatible with stream");
}
FSDataOutputStream out;
boolean ownStream = fileOption != null;
try {
if (ownStream) {
Path p = fileOption.getValue();
// this creates one entry in org.apache.hadoop.fs.FileSystem.CACHE
fs = FileSystem.newInstance(p.toUri(), conf);
int bufferSize = bufferSizeOption == null
? getBufferSize(conf)
: bufferSizeOption.getValue();
short replication = replicationOption == null
? fs.getDefaultReplication(p)
: (short) replicationOption.getValue();
long blockSize = blockSizeOption == null
? fs.getDefaultBlockSize(p)
: blockSizeOption.getValue();
if (appendIfExistsOption != null
&& appendIfExistsOption.getValue()
&& fs.exists(p)
&& hasIntactVersionHeader(p, fs)) {
// Read the file and verify header details
try (WALFile.Reader reader = new WALFile.Reader(
connectorConfig.getHadoopConfiguration(),
WALFile.Reader.file(p),
new Reader.OnlyHeaderOption()
)) {
if (reader.getVersion() != VERSION[3]) {
throw new VersionMismatchException(VERSION[3], reader.getVersion());
}
sync = reader.getSync();
}
out = fs.append(p, bufferSize);
this.appendMode = true;
} else {
out = fs.create(p, true, bufferSize, replication, blockSize);
}
} else {
out = streamOption.getValue();
}
init(connectorConfig, out, ownStream);
} catch (Exception re) {
log.warn("Failed creating a WAL Writer: " + re.getMessage());
if (fs != null) {
try {
//this deletes an entry from org.apache.hadoop.fs.FileSystem.CACHE
fs.close();
} catch (Throwable t) {
log.error("Could not close filesystem", t);
}
}
throw re;
}
}
private boolean hasIntactVersionHeader(Path p, FileSystem fs) throws IOException {
FileStatus[] statuses = fs.listStatus(p);
if (statuses.length != 1) {
throw new ConnectException("Expected exactly one log for WAL file " + p);
}
boolean result = statuses[0].getLen() >= VERSION.length;
if (!result) {
log.warn("Failed to read version header from WAL file " + p);
}
return result;
}
public static Option file(Path value) {
return new FileOption(value);
}
public static Option bufferSize(int value) {
return new BufferSizeOption(value);
}
public static Option stream(FSDataOutputStream value) {
return new StreamOption(value);
}
public static Option replication(short value) {
return new ReplicationOption(value);
}
public static Option appendIfExists(boolean value) {
return new AppendIfExistsOption(value);
}
public static Option blockSize(long value) {
return new BlockSizeOption(value);
}
void init(HdfsSinkConnectorConfig connectorConfig, FSDataOutputStream out, boolean ownStream)
throws IOException {
Configuration conf = connectorConfig.getHadoopConfiguration();
this.out = out;
this.ownOutputStream = ownStream;
SerializationFactory serializationFactory = new SerializationFactory(conf);
this.keySerializer = serializationFactory.getSerializer(WALEntry.class);
if (this.keySerializer == null) {
String errorMsg = String.format(
deserErrorFmt,
"Key",
WALEntry.class.getCanonicalName(),
CommonConfigurationKeys.IO_SERIALIZATIONS_KEY
);
throw new IOException(errorMsg);
}
this.keySerializer.open(buffer);
this.valSerializer = serializationFactory.getSerializer(WALEntry.class);
if (this.valSerializer == null) {
String errorMsg = String.format(
deserErrorFmt,
"Value",
WALEntry.class.getCanonicalName(),
CommonConfigurationKeys.IO_SERIALIZATIONS_KEY
);
throw new IOException(errorMsg);
}
this.valSerializer.open(buffer);
if (appendMode) {
sync();
} else {
writeFileHeader();
}
}
public synchronized void append(WALEntry key, WALEntry val)
throws IOException {
buffer.reset();
// Append the 'key'
keySerializer.serialize(key);
int keyLength = buffer.getLength();
if (keyLength < 0) {
throw new IOException("negative length keys not allowed: " + key);
}
valSerializer.serialize(val);
// Write the record out
checkAndWriteSync(); // sync
out.writeInt(buffer.getLength()); // total record length
out.writeInt(keyLength); // key portion length
out.write(buffer.getData(), 0, buffer.getLength()); // data
}
/**
* Returns the current length of the output file.
*
* <p>This always returns a synchronized position. In other words, immediately after calling
* {@link WALFile.Reader#seek(long)} with a position returned by this method, {@link
* WALFile.Reader#next(Writable)} may be called. However the key may be earlier in the file
* than key last written when this method was called (e.g., with block-compression, it may be
* the first key in the block that was being written when this method was called).
*
* @return the current length of the output file.
* @throws IOException Exception on getting position
*/
public synchronized long getLength() throws IOException {
return out.getPos();
}
private synchronized void checkAndWriteSync() throws IOException {
if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
// time to emit sync
sync();
}
}
private void writeFileHeader()
throws IOException {
out.write(VERSION); // write the version
out.write(sync); // write the sync bytes
out.flush(); // flush header
}
@Override
public synchronized void close() throws IOException {
try {
keySerializer.close();
valSerializer.close();
if (out != null) {
// Close the underlying stream iff we own it...
if (ownOutputStream) {
out.close();
} else {
out.flush();
}
}
} finally {
if (fs != null) {
try {
//this deletes an entry from org.apache.hadoop.fs.FileSystem.CACHE
fs.close();
} catch (Throwable t) {
log.error("Could not close FileSystem", t);
}
}
out = null;
}
}
@Override
public void sync() throws IOException {
if (sync != null && lastSyncPos != out.getPos()) {
out.writeInt(SYNC_ESCAPE); // mark the start of the sync
out.write(sync); // write sync
lastSyncPos = out.getPos(); // update lastSyncPos
}
}
@Override
public void hflush() throws IOException {
if (out != null) {
out.hflush();
}
}
@Override
public void hsync() throws IOException {
if (out != null) {
out.hsync();
}
}
/**
* A tag interface for all of the Reader options
*/
public interface Option {
}
static class FileOption extends Options.PathOption implements Option {
FileOption(Path path) {
super(path);
}
}
static class StreamOption extends Options.FSDataOutputStreamOption
implements Option {
StreamOption(FSDataOutputStream stream) {
super(stream);
}
}
static class BufferSizeOption extends Options.IntegerOption
implements Option {
BufferSizeOption(int value) {
super(value);
}
}
static class BlockSizeOption extends Options.LongOption implements Option {
BlockSizeOption(long value) {
super(value);
}
}
static class ReplicationOption extends Options.IntegerOption
implements Option {
ReplicationOption(int value) {
super(value);
}
}
static class AppendIfExistsOption extends Options.BooleanOption implements Option {
AppendIfExistsOption(boolean value) {
super(value);
}
}
}
public static class Reader implements java.io.Closeable {
private String filename;
private FileSystem fs;
private FSDataInputStream in;
private DataOutputBuffer outBuf = new DataOutputBuffer();
private byte version;
private byte[] sync = new byte[SYNC_HASH_SIZE];
private byte[] syncCheck = new byte[SYNC_HASH_SIZE];
private boolean syncSeen;
private long headerEnd;
private long end;
private int keyLength;
private int recordLength;
private Configuration conf;
private DataInputBuffer valBuffer = null;
private DataInputStream valIn = null;
private Deserializer<WALEntry> keyDeserializer;
private Deserializer<WALEntry> valDeserializer;
public Reader(Configuration conf, Option... opts) throws IOException {
// Look up the options, these are null if not set
FileOption fileOpt = Options.getOption(FileOption.class, opts);
InputStreamOption streamOpt = Options.getOption(InputStreamOption.class, opts);
LengthOption lenOpt = Options.getOption(LengthOption.class, opts);
BufferSizeOption bufOpt = Options.getOption(BufferSizeOption.class, opts);
// check for consistency
if ((fileOpt == null) == (streamOpt == null)) {
throw new
IllegalArgumentException("File or stream option must be specified");
}
if (fileOpt == null && bufOpt != null) {
throw new IllegalArgumentException("buffer size can only be set when a file is specified.");
}
// figure out the real values
Path filename = null;
FSDataInputStream file;
final long len;
try {
if (fileOpt != null) {
filename = fileOpt.getValue();
fs = FileSystem.newInstance(filename.toUri(),conf);
int bufSize = bufOpt == null ? getBufferSize(conf) : bufOpt.getValue();
len = null == lenOpt
? fs.getFileStatus(filename).getLen()
: lenOpt.getValue();
file = openFile(fs, filename, bufSize, len);
} else {
len = null == lenOpt ? Long.MAX_VALUE : lenOpt.getValue();
file = streamOpt.getValue();
}
StartOption startOpt = Options.getOption(StartOption.class, opts);
long start = startOpt == null ? 0 : startOpt.getValue();
// really set up
OnlyHeaderOption headerOnly = Options.getOption(OnlyHeaderOption.class, opts);
initialize(filename, file, start, len, conf, headerOnly != null);
} catch (RemoteException re) {
log.error("Failed creating a WAL Reader: " + re.getMessage());
if (fs != null) {
try {
fs.close();
} catch (Throwable t) {
log.error("Error closing FileSystem", t);
}
}
throw re;
}
}
/**
* Create an option to specify the path name of the sequence file.
*
* @param value the path to read
* @return a new option
*/
public static Option file(Path value) {
return new FileOption(value);
}
/**
* Create an option to specify the stream with the sequence file.
*
* @param value the stream to read.
* @return a new option
*/
public static Option stream(FSDataInputStream value) {
return new InputStreamOption(value);
}
/**
* Create an option to specify the starting byte to read.
*
* @param value the number of bytes to skip over
* @return a new option
*/
public static Option start(long value) {
return new StartOption(value);
}
/**
* Create an option to specify the number of bytes to read.
*
* @param value the number of bytes to read
* @return a new option
*/
public static Option length(long value) {
return new LengthOption(value);
}
/**
* Create an option with the buffer size for reading the given pathname.
*
* @param value the number of bytes to buffer
* @return a new option
*/
public static Option bufferSize(int value) {
return new BufferSizeOption(value);
}
/**
* Common work of the constructors.
*/
private void initialize(
Path filename, FSDataInputStream in,
long start, long length, Configuration conf,
boolean tempReader
) throws IOException {
if (in == null) {
throw new IllegalArgumentException("in == null");
}
this.filename = filename == null ? "<unknown>" : filename.toString();
this.in = in;
this.conf = conf;
boolean succeeded = false;
try {
seek(start);
this.end = this.in.getPos() + length;
// if it wrapped around, use the max
if (end < length) {
end = Long.MAX_VALUE;
}
init(tempReader);
succeeded = true;
} finally {
if (!succeeded) {
IOUtils.cleanup(log, this.in);
}
}
}
/**
* Override this method to specialize the type of {@link FSDataInputStream} returned.
*
* @param fs The file system used to open the file.
* @param file The file being read.
* @param bufferSize The buffer size used to read the file.
* @param length The length being read if it is equal to or greater than 0.
* Otherwise, the length is not available.
* @return The opened stream.
* @throws IOException Exception on opening file.
*/
protected FSDataInputStream openFile(
FileSystem fs, Path file,
int bufferSize, long length
) throws IOException {
return fs.open(file, bufferSize);
}
/**
* Initialize the {@link Reader}
*
* @param tempReader <code>true</code> if we are constructing a temporary and hence do not
* initialize every component; <code>false</code> otherwise.
* @throws IOException Exception on opening file.
*/
private void init(boolean tempReader) throws IOException {
byte[] versionBlock = new byte[VERSION.length];
in.readFully(versionBlock);
if ((versionBlock[0] != VERSION[0])
|| (versionBlock[1] != VERSION[1])
|| (versionBlock[2] != VERSION[2])
) {
throw new IOException(this + " not a WALFile");
}
// Set 'version'
version = versionBlock[3];
if (version > VERSION[3]) {
throw new VersionMismatchException(VERSION[3], version);
}
in.readFully(sync); // read sync bytes
headerEnd = in.getPos(); // record end of header
// Initialize... *not* if this we are constructing a temporary Reader
if (!tempReader) {
valBuffer = new DataInputBuffer();
valIn = valBuffer;
SerializationFactory serializationFactory =
new SerializationFactory(conf);
this.keyDeserializer =
getDeserializer(serializationFactory, WALEntry.class);
if (this.keyDeserializer == null) {
String errorMsg = String.format(
deserErrorFmt,
"Key",
WALEntry.class.getCanonicalName(),
CommonConfigurationKeys.IO_SERIALIZATIONS_KEY
);
throw new IOException(errorMsg);
}
this.keyDeserializer.open(valBuffer);
this.valDeserializer =
getDeserializer(serializationFactory, WALEntry.class);
if (this.valDeserializer == null) {
String errorMsg = String.format(
deserErrorFmt,
"Value",
WALEntry.class.getCanonicalName(),
CommonConfigurationKeys.IO_SERIALIZATIONS_KEY
);
throw new IOException(errorMsg);
}
this.valDeserializer.open(valIn);
}
}
private <T> Deserializer<T> getDeserializer(SerializationFactory sf, Class<T> c) {
return sf.getDeserializer(c);
}
private byte[] getSync() {
return sync;
}
/**
* Close the file.
*/
@Override
public synchronized void close() throws IOException {
try {
if (keyDeserializer != null) {
keyDeserializer.close();
}
if (valDeserializer != null) {
valDeserializer.close();
}
// Close the input-stream
in.close();
} finally {
try {
fs.close();
} catch (Throwable t) {
log.error("Unable to close FileSystem", t);
}
}
}
private byte getVersion() {
return version;
}
/**
* Returns the configuration used for this file.
*/
Configuration getConf() {
return conf;
}
/**
* Position valLenIn/valIn to the 'value' corresponding to the 'current' key
*/
private synchronized void seekToCurrentValue() throws IOException {
valBuffer.reset();
}
/**
* Get the 'value' corresponding to the last read 'key'.
*
* @param val : The 'value' to be read.
* @throws IOException Exception on reading key.
*/
public synchronized void getCurrentValue(Writable val)
throws IOException {
if (val instanceof Configurable) {
((Configurable) val).setConf(this.conf);
}
// Position stream to 'current' value
seekToCurrentValue();
val.readFields(valIn);
if (valIn.read() > 0) {
log.info("available bytes: " + valIn.available());
throw new IOException(
val
+ " read "
+ (valBuffer.getPosition() - keyLength)
+ " bytes, should read "
+ (valBuffer.getLength() - keyLength)
);
}
}
/**
* Get the 'value' corresponding to the last read 'key'.
*
* @param val : The 'value' to be read.
* @return the value corresponding to the last read key.
* @throws IOException Exception on reading key.
*/
public synchronized WALEntry getCurrentValue(WALEntry val)
throws IOException {
if (val instanceof Configurable) {
((Configurable) val).setConf(this.conf);
}
// Position stream to 'current' value
seekToCurrentValue();
val = deserializeValue(val);
if (valIn.read() > 0) {
log.info("available bytes: " + valIn.available());
throw new IOException(
val
+ " read "
+ (valBuffer.getPosition() - keyLength)
+ " bytes, should read "
+ (valBuffer.getLength() - keyLength)
);
}
return val;
}
private WALEntry deserializeValue(WALEntry val) throws IOException {
return valDeserializer.deserialize(val);
}
/**
* Read and return the next record length, potentially skipping over a sync block.
*
* @return the length of the next record or -1 if there is no next record
*/
private synchronized int readRecordLength() throws IOException {
if (in.getPos() >= end) {
return -1;
}
int length = in.readInt();
if (sync != null && length == SYNC_ESCAPE) {
// process a sync entry
// read syncCheck
in.readFully(syncCheck);
if (!Arrays.equals(sync, syncCheck)) {
// check it
throw new CorruptWalFileException("File is corrupt!");
}
syncSeen = true;
if (in.getPos() >= end) {
return -1;
}
length = in.readInt(); // re-read length
} else {
syncSeen = false;
}
return length;
}
/**
* Read the next key in the file into <code>key</code>, skipping its value. True if another
* entry exists, and false at end of file.
*
* @param key the writable to read the key into
* @return whether another key exists after reading a key
* @throws IOException Exception on reading key.
*/
public synchronized boolean next(Writable key) throws IOException {
if (key.getClass() != WALEntry.class) {
throw new IOException("wrong key class: " + key.getClass().getName()
+ " is not " + WALEntry.class);
}
outBuf.reset();
keyLength = next(outBuf);
if (keyLength < 0) {
return false;
}
valBuffer.reset(outBuf.getData(), outBuf.getLength());
key.readFields(valBuffer);
valBuffer.mark(0);
if (valBuffer.getPosition() != keyLength) {
throw new IOException(key + " read " + valBuffer.getPosition()
+ " bytes, should read " + keyLength);
}
return true;
}
/**
* Read the next key/value pair in the file into <code>key</code> and <code>val</code>. Returns
* true if such a pair exists and false when at end of file
*
* @param key the writable to read the key into
* @param val the writable to read the val into
* @return whether another key value pair exists after reading a key
* @throws IOException Exception on reading key pair.
*/
public synchronized boolean next(Writable key, Writable val) throws IOException {
if (val.getClass() != WALEntry.class) {
throw new IOException("wrong value class: " + val + " is not " + WALEntry.class);
}
boolean more = next(key);
if (more) {
getCurrentValue(val);
}
return more;
}
/**
* Read the next key/value pair in the file into <code>buffer</code>. Returns the length of the
* key read, or -1 if at end of file. The length of the value may be computed by calling
* buffer.getLength() before and after calls to this method.
*/
synchronized int next(DataOutputBuffer buffer) throws IOException {
try {
int length = readRecordLength();
if (length == -1) {
return -1;
}
int keyLength = in.readInt();
buffer.write(in, length);
return keyLength;
} catch (ChecksumException e) { // checksum failure
handleChecksumException(e);
return next(buffer);
}
}
/**
* Read the next key in the file, skipping its value. Return null at end of file.
*
* @param key the current WALEntry
* @return null at the end of file
* @throws IOException Exception on reading key.
*/
public synchronized WALEntry next(WALEntry key) throws IOException {
outBuf.reset();
keyLength = next(outBuf);
if (keyLength < 0) {
return null;
}
valBuffer.reset(outBuf.getData(), outBuf.getLength());
key = deserializeKey(key);
valBuffer.mark(0);
if (valBuffer.getPosition() != keyLength) {
throw new IOException(key + " read " + valBuffer.getPosition()
+ " bytes, should read " + keyLength);
}
return key;
}
private WALEntry deserializeKey(WALEntry key) throws IOException {
return keyDeserializer.deserialize(key);
}
private void handleChecksumException(ChecksumException e)
throws IOException {
if (this.conf.getBoolean("io.skip.checksum.errors", false)) {
log.warn("Bad checksum at " + getPosition() + ". Skipping entries.");
sync(getPosition() + this.conf.getInt("io.bytes.per.checksum", 512));
} else {
throw e;
}
}
/**
* disables sync. often invoked for tmp files
*/
synchronized void ignoreSync() {
sync = null;
}
/**
* Set the current byte position in the input file.
*
* <p>The position passed must be a position returned by {@link WALFile.Writer#getLength()} when
* writing this file. To seek to an arbitrary position, use {@link WALFile.Reader#sync(long)}.
*
* @param position a position returned by {@link WALFile.Writer#getLength()} whem
* writing this file.
* @throws IOException Exception on setting byte position
*/
public synchronized void seek(long position) throws IOException {
in.seek(position);
}
/**
* Seek to the start of the file after the header.
*
* @throws IOException if unable to seek to the end of the header
*/
public void seekToFirstRecord() throws IOException {
in.seek(headerEnd);
}
/**
* Seek to the next sync mark past a given position.
*
* @param position sync mark will be found past the given position.
* @throws IOException Exception on setting byte position.
*/
public synchronized void sync(long position) throws IOException {
if (position + SYNC_SIZE >= end) {
seek(end);
return;
}
if (position < headerEnd) {
// seek directly to first record
in.seek(headerEnd);
// note the sync marker "seen" in the header
syncSeen = true;
return;
}
try {
seek(position + 4); // skip escape
in.readFully(syncCheck);
int syncLen = sync.length;
for (int i = 0; in.getPos() < end; i++) {
int j = 0;
for (; j < syncLen; j++) {
if (sync[j] != syncCheck[(i + j) % syncLen]) {
break;
}
}
if (j == syncLen) {
in.seek(in.getPos() - SYNC_SIZE); // position before sync
return;
}
syncCheck[i % syncLen] = in.readByte();
}
} catch (ChecksumException e) { // checksum failure
handleChecksumException(e);
}
}
/**
* Returns true iff the previous call to next passed a sync mark.
* @return true iff the previous call to next passed a sync mark.
*/
public synchronized boolean syncSeen() {
return syncSeen;
}
/**
* Return the current byte position in the input file.
* @return the current byte position in the input file.
* @throws IOException Exception on getting position.
*/
public synchronized long getPosition() throws IOException {
return in.getPos();
}
/**