-
Notifications
You must be signed in to change notification settings - Fork 1
/
Copy pathFIX_EOF
1266 lines (1194 loc) · 52 KB
/
FIX_EOF
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
diff --git a/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterProvider.java b/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterProvider.java
index 401aff3..98b60f2 100644
--- a/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterProvider.java
+++ b/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterProvider.java
@@ -1,11 +1,12 @@
package com.linkedin.camus.etl;
-import com.linkedin.camus.coders.CamusWrapper;
import java.io.IOException;
-import org.apache.hadoop.mapreduce.RecordWriter;
+
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import com.linkedin.camus.coders.CamusWrapper;
+
/**
*
*
@@ -14,7 +15,8 @@ public interface RecordWriterProvider {
String getFilenameExtension();
- RecordWriter<IEtlKey, CamusWrapper> getDataRecordWriter(
+ @SuppressWarnings("rawtypes")
+ RecordWriterWithCloseStatus<IEtlKey, CamusWrapper> getDataRecordWriter(
TaskAttemptContext context, String fileName, CamusWrapper data, FileOutputCommitter committer) throws IOException,
InterruptedException;
-}
+ }
diff --git a/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterWithCloseStatus.java b/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterWithCloseStatus.java
new file mode 100644
index 0000000..2496d61
--- /dev/null
+++ b/camus-api/src/main/java/com/linkedin/camus/etl/RecordWriterWithCloseStatus.java
@@ -0,0 +1,19 @@
+package com.linkedin.camus.etl;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+
+
+/**
+ * This is writer interface added to check if underlying writer is closed or not.
+ *
+ * @param <K> key
+ * @param <V> value
+ */
+public abstract class RecordWriterWithCloseStatus<K, V> extends RecordWriter<K, V>{
+ /**
+ * Give Ability to check if close has been called on the writer or File has been closed on not..
+ * @return
+ */
+ public abstract boolean isClose();
+
+}
\ No newline at end of file
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/CamusJob.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/CamusJob.java
index e9662f1..20dda22 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/CamusJob.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/CamusJob.java
@@ -1,16 +1,5 @@
package com.linkedin.camus.etl.kafka;
-import com.linkedin.camus.etl.kafka.common.DateUtils;
-import com.linkedin.camus.etl.kafka.common.EtlCounts;
-import com.linkedin.camus.etl.kafka.common.EtlKey;
-import com.linkedin.camus.etl.kafka.common.ExceptionWritable;
-import com.linkedin.camus.etl.kafka.common.Source;
-import com.linkedin.camus.etl.kafka.mapred.EtlInputFormat;
-import com.linkedin.camus.etl.kafka.mapred.EtlMapper;
-import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
-import com.linkedin.camus.etl.kafka.mapred.EtlRecordReader;
-import com.linkedin.camus.etl.kafka.reporter.BaseReporter;
-import com.linkedin.camus.etl.kafka.reporter.TimeReporter;
import java.io.BufferedReader;
import java.io.File;
@@ -18,23 +7,16 @@ import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
-import java.lang.ClassNotFoundException;
import java.lang.reflect.InvocationTargetException;
-import java.net.URI;
import java.net.URISyntaxException;
-import java.text.NumberFormat;
import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Arrays;
+import java.util.Comparator;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.Comparator;
-import java.util.Arrays;
import java.util.regex.Pattern;
import org.apache.commons.cli.CommandLine;
@@ -55,7 +37,6 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobID;
import org.apache.hadoop.mapred.TIPStatus;
import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapred.TaskReport;
@@ -80,6 +61,16 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormatter;
+import com.linkedin.camus.etl.kafka.common.DateUtils;
+import com.linkedin.camus.etl.kafka.common.EtlCounts;
+import com.linkedin.camus.etl.kafka.common.EtlKey;
+import com.linkedin.camus.etl.kafka.common.ExceptionWritable;
+import com.linkedin.camus.etl.kafka.common.Source;
+import com.linkedin.camus.etl.kafka.mapred.EtlInputFormat;
+import com.linkedin.camus.etl.kafka.mapred.EtlMapper;
+import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
+import com.linkedin.camus.etl.kafka.mapred.EtlRecordReader;
+import com.linkedin.camus.etl.kafka.reporter.BaseReporter;
public class CamusJob extends Configured implements Tool {
@@ -119,7 +110,6 @@ public class CamusJob extends Configured implements Tool {
private Job hadoopJob = null;
private final Properties props;
-
private DateTimeFormatter dateFmt = DateUtils.getDateTimeFormatter("YYYY-MM-dd-HH-mm-ss", DateTimeZone.UTC);
public CamusJob() throws IOException {
@@ -132,7 +122,7 @@ public class CamusJob extends Configured implements Tool {
public CamusJob(Properties props, Logger log) throws IOException {
this.props = props;
- this.log = log;
+ CamusJob.log = log;
}
private static HashMap<String, Long> timingMap = new HashMap<String, Long>();
@@ -203,7 +193,6 @@ public class CamusJob extends Configured implements Tool {
break;
}
}
-
if (!filterMatch)
DistributedCache.addFileToClassPath(status[i].getPath(), conf, fs);
}
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/AvroRecordWriterProvider.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/AvroRecordWriterProvider.java
index a0f25c5..299c1c0 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/AvroRecordWriterProvider.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/AvroRecordWriterProvider.java
@@ -1,67 +1,103 @@
package com.linkedin.camus.etl.kafka.common;
-import com.linkedin.camus.coders.CamusWrapper;
-import com.linkedin.camus.etl.IEtlKey;
-import com.linkedin.camus.etl.RecordWriterProvider;
-import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
import java.io.IOException;
+
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import com.linkedin.camus.coders.CamusWrapper;
+import com.linkedin.camus.etl.IEtlKey;
+import com.linkedin.camus.etl.RecordWriterProvider;
+import com.linkedin.camus.etl.RecordWriterWithCloseStatus;
+import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
+
+import org.apache.log4j.Logger;
+
/**
*
*
*/
public class AvroRecordWriterProvider implements RecordWriterProvider {
- public final static String EXT = ".avro";
-
- public AvroRecordWriterProvider(TaskAttemptContext context) {
- }
+ public final static String EXT = ".avro";
+
+ private static Logger log = Logger.getLogger(AvroRecordWriterProvider.class);
+
- @Override
- public String getFilenameExtension() {
- return EXT;
- }
+ public AvroRecordWriterProvider(TaskAttemptContext context) {
+ }
+
+ @Override
+ public String getFilenameExtension() {
+ return EXT;
+ }
- @Override
- public RecordWriter<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
- CamusWrapper data, FileOutputCommitter committer) throws IOException, InterruptedException {
- final DataFileWriter<Object> writer = new DataFileWriter<Object>(new SpecificDatumWriter<Object>());
+ @SuppressWarnings("rawtypes")
+ @Override
+ public RecordWriterWithCloseStatus<IEtlKey, CamusWrapper> getDataRecordWriter(
+ TaskAttemptContext context,
+ String fileName,
+ CamusWrapper data,
+ FileOutputCommitter committer) throws IOException, InterruptedException {
+ final DataFileWriter<Object> writer = new DataFileWriter<Object>(
+ new SpecificDatumWriter<Object>());
- if (FileOutputFormat.getCompressOutput(context)) {
- if ("snappy".equals(EtlMultiOutputFormat.getEtlOutputCodec(context))) {
- writer.setCodec(CodecFactory.snappyCodec());
- } else {
- int level = EtlMultiOutputFormat.getEtlDeflateLevel(context);
- writer.setCodec(CodecFactory.deflateCodec(level));
- }
- }
+ if (FileOutputFormat.getCompressOutput(context)) {
+ if ("snappy".equals(EtlMultiOutputFormat.getEtlOutputCodec(context))) {
+ writer.setCodec(CodecFactory.snappyCodec());
+ } else {
+ int level = EtlMultiOutputFormat.getEtlDeflateLevel(context);
+ writer.setCodec(CodecFactory.deflateCodec(level));
+ }
+ }
- Path path = committer.getWorkPath();
- path = new Path(path, EtlMultiOutputFormat.getUniqueFile(context, fileName, EXT));
- writer.create(((GenericRecord) data.getRecord()).getSchema(), path.getFileSystem(context.getConfiguration())
- .create(path));
+ Path path = committer.getWorkPath();
+ path = new Path(path, EtlMultiOutputFormat.getUniqueFile(context, fileName, EXT));
+ writer.create(((GenericRecord) data.getRecord()).getSchema(),
+ path.getFileSystem(context.getConfiguration()).create(path));
- writer.setSyncInterval(EtlMultiOutputFormat.getEtlAvroWriterSyncInterval(context));
+ writer.setSyncInterval(EtlMultiOutputFormat.getEtlAvroWriterSyncInterval(context));
- return new RecordWriter<IEtlKey, CamusWrapper>() {
- @Override
- public void write(IEtlKey ignore, CamusWrapper data) throws IOException {
- writer.append(data.getRecord());
- }
+ return new RecordWriterWithCloseStatus<IEtlKey, CamusWrapper>() {
+
+ private volatile boolean close;
+ @Override
+ public void write(IEtlKey ignore, CamusWrapper data) throws IOException {
+ writer.append(data.getRecord());
+ }
- @Override
- public void close(TaskAttemptContext arg0) throws IOException, InterruptedException {
- writer.close();
- }
- };
- }
+ @Override
+ public void close(TaskAttemptContext arg0) throws IOException, InterruptedException {
+ writer.close();
+ close = true;
+ }
+
+ /**
+ * THis is our last attemp to close the file...
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ if(!this.close){
+ log.error("This file was not closed so try to close during the JVM finalize..");
+ try{
+ writer.close();
+ }catch(Throwable th){
+ log.error("File Close erorr during finalize()");
+ }
+ }
+ super.finalize();
+ }
+
+ @Override
+ public boolean isClose() {
+ return close;
+ }
+ };
+ }
}
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/SequenceFileRecordWriterProvider.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/SequenceFileRecordWriterProvider.java
index b1af571..6d84cce 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/SequenceFileRecordWriterProvider.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/SequenceFileRecordWriterProvider.java
@@ -1,28 +1,26 @@
package com.linkedin.camus.etl.kafka.common;
-import com.linkedin.camus.coders.CamusWrapper;
-import com.linkedin.camus.etl.IEtlKey;
-import com.linkedin.camus.etl.RecordWriterProvider;
-import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
import java.io.IOException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
-
-import org.apache.hadoop.conf.Configuration;
-
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
import org.apache.log4j.Logger;
+import com.linkedin.camus.coders.CamusWrapper;
+import com.linkedin.camus.etl.IEtlKey;
+import com.linkedin.camus.etl.RecordWriterProvider;
+import com.linkedin.camus.etl.RecordWriterWithCloseStatus;
+import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
/**
* Provides a RecordWriter that uses SequenceFile.Writer to write
@@ -35,72 +33,103 @@ import org.apache.log4j.Logger;
*
*/
public class SequenceFileRecordWriterProvider implements RecordWriterProvider {
- public static final String ETL_OUTPUT_RECORD_DELIMITER = "etl.output.record.delimiter";
- public static final String DEFAULT_RECORD_DELIMITER = "";
-
- private static Logger log = Logger.getLogger(SequenceFileRecordWriterProvider.class);
+ public static final String ETL_OUTPUT_RECORD_DELIMITER = "etl.output.record.delimiter";
+ public static final String DEFAULT_RECORD_DELIMITER = "";
- protected String recordDelimiter = null;
+ private static Logger log = Logger.getLogger(SequenceFileRecordWriterProvider.class);
- public SequenceFileRecordWriterProvider(TaskAttemptContext context) {
-
- }
-
- // TODO: Make this configurable somehow.
- // To do this, we'd have to make SequenceFileRecordWriterProvider have an
- // init(JobContext context) method signature that EtlMultiOutputFormat would always call.
- @Override
- public String getFilenameExtension() {
- return "";
- }
-
- @Override
- public RecordWriter<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
- CamusWrapper camusWrapper, FileOutputCommitter committer) throws IOException, InterruptedException {
-
- Configuration conf = context.getConfiguration();
+ protected String recordDelimiter = null;
+
+ public SequenceFileRecordWriterProvider(TaskAttemptContext context) {
+
+ }
- // If recordDelimiter hasn't been initialized, do so now
- if (recordDelimiter == null) {
- recordDelimiter = conf.get(ETL_OUTPUT_RECORD_DELIMITER, DEFAULT_RECORD_DELIMITER);
+ // TODO: Make this configurable somehow.
+ // To do this, we'd have to make SequenceFileRecordWriterProvider have an
+ // init(JobContext context) method signature that EtlMultiOutputFormat would always call.
+ @Override
+ public String getFilenameExtension() {
+ return "";
}
- CompressionCodec compressionCodec = null;
- CompressionType compressionType = CompressionType.NONE;
+ @SuppressWarnings("rawtypes")
+ @Override
+ public RecordWriterWithCloseStatus<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
+ CamusWrapper camusWrapper, FileOutputCommitter committer) throws IOException, InterruptedException {
- // Determine compression type (BLOCK or RECORD) and compression codec to use.
- if (SequenceFileOutputFormat.getCompressOutput(context)) {
- compressionType = SequenceFileOutputFormat.getOutputCompressionType(context);
- Class<?> codecClass = SequenceFileOutputFormat.getOutputCompressorClass(context, DefaultCodec.class);
- // Instantiate the CompressionCodec Class
- compressionCodec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, conf);
- }
+ Configuration conf = context.getConfiguration();
- // Get the filename for this RecordWriter.
- Path path =
- new Path(committer.getWorkPath(), EtlMultiOutputFormat.getUniqueFile(context, fileName, getFilenameExtension()));
-
- log.info("Creating new SequenceFile.Writer with compression type " + compressionType + " and compression codec "
- + (compressionCodec != null ? compressionCodec.getClass().getName() : "null"));
- final SequenceFile.Writer writer =
- SequenceFile.createWriter(path.getFileSystem(conf), conf, path, LongWritable.class, Text.class,
- compressionType, compressionCodec, context);
-
- // Return a new anonymous RecordWriter that uses the
- // SequenceFile.Writer to write data to HDFS
- return new RecordWriter<IEtlKey, CamusWrapper>() {
- @Override
- public void write(IEtlKey key, CamusWrapper data) throws IOException, InterruptedException {
- String record = (String) data.getRecord() + recordDelimiter;
- // Use the timestamp from the EtlKey as the key for this record.
- // TODO: Is there a better key to use here?
- writer.append(new LongWritable(key.getTime()), new Text(record));
+ // If recordDelimiter hasn't been initialized, do so now
+ if (recordDelimiter == null) {
+ recordDelimiter = conf.get(ETL_OUTPUT_RECORD_DELIMITER, DEFAULT_RECORD_DELIMITER);
}
- @Override
- public void close(TaskAttemptContext context) throws IOException, InterruptedException {
- writer.close();
+ CompressionCodec compressionCodec = null;
+ CompressionType compressionType = CompressionType.NONE;
+
+ // Determine compression type (BLOCK or RECORD) and compression codec to use.
+ if (SequenceFileOutputFormat.getCompressOutput(context)) {
+ compressionType = SequenceFileOutputFormat.getOutputCompressionType(context);
+ Class<?> codecClass = SequenceFileOutputFormat.getOutputCompressorClass(context, DefaultCodec.class);
+ // Instantiate the CompressionCodec Class
+ compressionCodec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, conf);
}
- };
- }
+
+ // Get the filename for this RecordWriter.
+ Path path =
+ new Path(committer.getWorkPath(), EtlMultiOutputFormat.getUniqueFile(context, fileName, getFilenameExtension()));
+
+ log.info("Creating new SequenceFile.Writer with compression type " + compressionType + " and compression codec "
+ + (compressionCodec != null ? compressionCodec.getClass().getName() : "null"));
+ final SequenceFile.Writer writer =
+ SequenceFile.createWriter(path.getFileSystem(conf), conf, path, LongWritable.class, Text.class,
+ compressionType, compressionCodec, context);
+ // Return a new anonymous RecordWriter that uses the
+ // SequenceFile.Writer to write data to HDFS
+ return new RecordWriterWithCloseStatus<IEtlKey, CamusWrapper>() {
+ private volatile boolean close;
+
+ @Override
+ public void write(IEtlKey key, CamusWrapper data) throws IOException, InterruptedException {
+
+ String record = (String)data.getRecord();
+ // do not do string concatenate if not needed...here...???
+ if(recordDelimiter != null && !recordDelimiter.isEmpty()){
+ record = record + recordDelimiter;
+ }
+
+ /**
+ * What if file is closed ? Should we create a new one here..?
+ * should we reopen it or not...
+ */
+ // Use the timestamp from the EtlKey as the key for this record.
+ // TODO: Is there a better key to use here?
+ writer.append(new LongWritable(key.getTime()), new Text(record));
+ }
+
+ @Override
+ public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+ writer.close();
+ close = true;
+ }
+ @Override
+ protected void finalize() throws Throwable {
+ if(!this.close){
+ log.error("This file was not closed so try to close during the JVM finalize..");
+ try{
+ writer.close();
+ }catch(Throwable th){
+ log.error("File Close erorr during finalize()");
+ }
+ }
+ super.finalize();
+ }
+
+ @Override
+ public boolean isClose() {
+ return close;
+ }
+ };
+ }
+
}
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/StringRecordWriterProvider.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/StringRecordWriterProvider.java
index 4535564..1b058cd 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/StringRecordWriterProvider.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/common/StringRecordWriterProvider.java
@@ -1,29 +1,28 @@
package com.linkedin.camus.etl.kafka.common;
-import com.linkedin.camus.coders.CamusWrapper;
-import com.linkedin.camus.etl.IEtlKey;
-import com.linkedin.camus.etl.RecordWriterProvider;
-import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
-
import java.io.DataOutputStream;
import java.io.IOException;
-import org.apache.avro.file.CodecFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.SnappyCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.log4j.Logger;
+import com.linkedin.camus.coders.CamusWrapper;
+import com.linkedin.camus.etl.IEtlKey;
+import com.linkedin.camus.etl.RecordWriterProvider;
+import com.linkedin.camus.etl.RecordWriterWithCloseStatus;
+import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat;
+
/**
* Provides a RecordWriter that uses FSDataOutputStream to write
@@ -32,6 +31,7 @@ import org.apache.log4j.Logger;
public class StringRecordWriterProvider implements RecordWriterProvider {
public static final String ETL_OUTPUT_RECORD_DELIMITER = "etl.output.record.delimiter";
public static final String DEFAULT_RECORD_DELIMITER = "\n";
+ private static Logger log = Logger.getLogger(StringRecordWriterProvider.class);
protected String recordDelimiter = null;
@@ -71,7 +71,7 @@ public class StringRecordWriterProvider implements RecordWriterProvider {
}
@Override
- public RecordWriter<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
+ public RecordWriterWithCloseStatus<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
CamusWrapper camusWrapper, FileOutputCommitter committer) throws IOException, InterruptedException {
// If recordDelimiter hasn't been initialized, do so now
@@ -112,28 +112,48 @@ public class StringRecordWriterProvider implements RecordWriterProvider {
};
*/
}
+ @SuppressWarnings("rawtypes")
+ protected static class ByteRecordWriter extends RecordWriterWithCloseStatus<IEtlKey, CamusWrapper> {
+
+ private volatile boolean close;
+ private DataOutputStream out;
+ private String recordDelimiter;
+
+ public ByteRecordWriter(DataOutputStream out, String recordDelimiter) {
+ this.out = out;
+ this.recordDelimiter = recordDelimiter;
+ }
- protected static class ByteRecordWriter extends RecordWriter<IEtlKey, CamusWrapper> {
- private DataOutputStream out;
- private String recordDelimiter;
-
- public ByteRecordWriter(DataOutputStream out, String recordDelimiter) {
- this.out = out;
- this.recordDelimiter = recordDelimiter;
- }
-
- @Override
- public void write(IEtlKey ignore, CamusWrapper value) throws IOException {
- boolean nullValue = value == null;
- if (!nullValue) {
- String record = (String) value.getRecord() + recordDelimiter;
- out.write(record.getBytes());
+ @Override
+ public void write(IEtlKey ignore, CamusWrapper value) throws IOException {
+ boolean nullValue = value == null;
+ if (!nullValue) {
+ String record = (String)value.getRecord() + recordDelimiter;
+ out.write(record.getBytes());
+ }
}
- }
- @Override
- public void close(TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
- out.close();
- }
+ @Override
+ public void close(TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+ out.close();
+ close = true;
+ }
+
+ protected void finalize() throws Throwable {
+ if(!this.close){
+ log.error("This file was not closed so try to close during the JVM finalize..");
+ try{
+ out.close();
+ }catch(Throwable th){
+ log.error("File Close erorr during finalize()");
+ }
+ }
+ super.finalize();
+ }
+
+ @Override
+ public boolean isClose() {
+ return close;
+ }
}
}
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlInputFormat.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlInputFormat.java
index 2edd4f8..377b1d5 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlInputFormat.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlInputFormat.java
@@ -1,16 +1,5 @@
package com.linkedin.camus.etl.kafka.mapred;
-import com.linkedin.camus.coders.CamusWrapper;
-import com.linkedin.camus.coders.MessageDecoder;
-import com.linkedin.camus.etl.kafka.CamusJob;
-import com.linkedin.camus.etl.kafka.coders.KafkaAvroMessageDecoder;
-import com.linkedin.camus.etl.kafka.coders.MessageDecoderFactory;
-import com.linkedin.camus.etl.kafka.common.EtlKey;
-import com.linkedin.camus.etl.kafka.common.EtlRequest;
-import com.linkedin.camus.etl.kafka.common.LeaderInfo;
-import com.linkedin.camus.workallocater.CamusRequest;
-import com.linkedin.camus.workallocater.WorkAllocator;
-
import java.io.IOException;
import java.net.URI;
import java.security.InvalidParameterException;
@@ -25,7 +14,6 @@ import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;
import kafka.api.PartitionOffsetRequestInfo;
@@ -69,6 +57,7 @@ import com.linkedin.camus.workallocater.WorkAllocator;
/**
* Input format for a Kafka pull job.
*/
+@SuppressWarnings("rawtypes")
public class EtlInputFormat extends InputFormat<EtlKey, CamusWrapper> {
public static final String KAFKA_BLACKLIST_TOPIC = "kafka.blacklist.topics";
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputCommitter.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputCommitter.java
index 7ea80b1..9f99f21 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputCommitter.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputCommitter.java
@@ -5,13 +5,16 @@ import java.io.IOException;
import java.io.OutputStream;
import java.lang.reflect.Constructor;
import java.util.ArrayList;
+import java.util.Date;
import java.util.HashMap;
import java.util.Map;
+import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
@@ -22,8 +25,10 @@ import org.apache.log4j.Logger;
import org.codehaus.jackson.map.ObjectMapper;
import com.linkedin.camus.etl.RecordWriterProvider;
+import com.linkedin.camus.etl.kafka.CamusJob;
import com.linkedin.camus.etl.kafka.common.EtlCounts;
import com.linkedin.camus.etl.kafka.common.EtlKey;
+import com.linkedin.camus.etl.kafka.mapred.RetryLogic.Delegate;
public class EtlMultiOutputCommitter extends FileOutputCommitter {
@@ -41,7 +46,17 @@ public class EtlMultiOutputCommitter extends FileOutputCommitter {
if (! fs.exists(path.getParent())) {
mkdirs(fs, path.getParent());
}
- fs.mkdirs(path);
+ boolean result = fs.mkdirs(path);
+ /*
+ * We do not want silent failure...
+ * SHould we re-check if fs.exists(path.getParent()) then alert ...? Should we trust FS or double check...?
+ * Based on feedback please refer to https://github.com/linkedin/camus/issues/189
+ */
+ if(!result){
+ String msg = "Unable to create directory: " + path.toString();
+ log.error(msg);
+ throw new IOException(msg);
+ }
}
public void addCounts(EtlKey key) throws IOException {
@@ -148,9 +163,233 @@ public class EtlMultiOutputCommitter extends FileOutputCommitter {
super.commitTask(context);
}
- protected void commitFile(JobContext job, Path source, Path target) throws IOException {
- FileSystem.get(job.getConfiguration()).rename(source, target);
- }
+ protected void commitFile(final JobContext job, final Path source, final Path target) throws IOException{
+
+ /**
+ * Custom code to commit the file across volumes as suggested by the Peter Newcomb...
+ * We also wanted to preserve the so we need attempted task id as well... hence this will be only called when task that is needs to be committed.
+ * mapred.map.tasks.speculative.execution true If true, then multiple instances of some map tasks may be executed in parallel for safe reason add "taskid to temp"
+ *
+ * user volume/tmp (volume 1) to /raw/logmon (volume 2) and then rename file to final target location (/raw/logmon/...)
+ * can not remove /_temporary as other camus job may write to it so delete it till taskid.
+ *
+ * This is very inefficient solution (Kakfa ->copy to temp user volume-> copy to temp day volume ->rename to final destination.
+ * This will increase the latency of Camus job, Hence, lags is expected to go high.
+ *
+ */
+
+
+ if(EtlMultiOutputFormat.isCopyAndRenameFinalDestination(job)){
+ long startTime = System.currentTimeMillis();
+ log.info("Starting copy-rename-commit at " + new Date());
+ CamusJob.startTiming("copy-rename-commit");
+ log.info("Using the Two Steps Process Copy to Temp and then rename to target.");
+ final TaskAttemptContext taskContext = (TaskAttemptContext) job;
+ // TODO move this to configuration....this is not requirement
+ final int retry=3;
+ Path parentpath = target.getParent();
+ // lets create a temp directory....
+ //TODO...delete this directories...
+ //TODO create mapper _temporary dir with mapper task so it does not show up....
+ final String tempLocation = parentpath.toString() + "/_temporary/_"+taskContext.getTaskAttemptID().toString();
+ final FileSystem fs = FileSystem.get(context.getConfiguration());
+
+ try{
+ //TODO file needs to compare the file status to destination or md5dum so we do not get corruption or eof issue...
+ //final FileStatus fileInfo = fs.getFileStatus(source);
+
+ //TODO get this from configuration....
+ RetryLogic<Boolean> retryFileCheck = new RetryLogic<Boolean>(retry, 1, TimeUnit.SECONDS, this.log);
+
+ final Delegate<Boolean> tempDirExists = new RetryLogic.Delegate<Boolean>() {
+ @Override
+ public Boolean call() throws Exception {
+ final Path destinationTemp = new Path(tempLocation);
+ boolean result = false;
+
+ // for ANY fatal underlying FS erorr try 3 more times...
+ // try to create a temp directory try 3 times if there is any failure then do not try...
+ while(retry > 0 && !result){
+ result = fs.exists(destinationTemp);
+ if(!result){
+ result = fs.mkdirs(destinationTemp);
+ }
+ }
+ return result;
+ }
+
+ };
+ boolean tempDirectoryCreated = false;
+ try {
+ tempDirectoryCreated = retryFileCheck.getResult(tempDirExists);
+ } catch (Exception e) {
+ log.error("Could not commit file due to error exit....",e);
+ // this will cause entire Mapper task to fail so we can start from last offset again three for no data loss ???
+ throw new RuntimeException("Temp Folder Creation failed...",e);
+ }
+
+ if(tempDirectoryCreated){
+ //now try to move this file to temp folder...
+
+ try {
+ final CopyToTempTask copyToTempDir = new CopyToTempTask(tempLocation, source, fs, log);
+ // now we have copied to respective volume so let try to rename it....this should not fail at all ....
+ final boolean copyToTempFolderFile = retryFileCheck.getResult(copyToTempDir);
+ if(copyToTempFolderFile){
+ final Delegate<Boolean> renameFromTempToFinalDestination = new RetryLogic.Delegate<Boolean>() {
+ @Override
+ public Boolean call() throws Exception {
+ int retry = 3;
+ boolean result = false;
+ do{
+ try{
+ result = fs.rename(copyToTempDir.getTempPath(),target);
+ if(!result){
+ log.error("Failed to commit the file to final destination ="+target.toString() + " and temp path="+copyToTempDir.getTempPath().toString());
+ if(!fs.exists(copyToTempDir.getTempPath())){
+ // should we retry copy cmd here ...?
+ log.error("Final temp path="+copyToTempDir.getTempPath().toString() + " does not exits so giveup.");
+ break;
+ }
+ }
+
+ }catch(IOException e){
+ log.error("IOException while rename to final target name = "+target.getName(),e);
+ }
+ retry--;
+ }while(!result && retry > 0);
+ return result;
+ }
+
+ };
+
+ final boolean finalResult = retryFileCheck.getResult(renameFromTempToFinalDestination);
+ if(finalResult){
+ // it is ok if it does not get deleted since we have directory clean up will take care of it...
+ boolean delte = fs.delete(source,true);
+ if(!delte){
+ fs.deleteOnExit(source);
+ }
+ }else{
+ String message = "File can not be commited at this movement so lets try next time due to underlying FileSystem erorr. Error is trown so offset does not advance, hence no data loss.\n"+
+ "source=" + source.toString() + " \n" +
+ "final target= " + target.toString() + "\n";
+
+ log.error(message);
+ throw new RuntimeException(message);
+ }
+ }
+ } catch (Exception e) {
+ if(e instanceof RuntimeException){
+ log.error("Giving up commit rename since the copy failed" + source.toString() +" and to=" + tempLocation,e);
+ throw (RuntimeException)e;
+ }else{
+ log.error("Giving up commit rename since the copy failed" + source.toString() +" and to=" + tempLocation,e);
+ throw new RuntimeException("Giving up commit since the copy failed" + source.toString() +" and to=" + tempLocation,e);
+ }
+ }
+ }
+
+ }finally{
+ // finally delete the temp location....
+ //partition_month_utc=2015-03/partition_day_utc=2015-03-11/partition_minute_bucket=2015-03-11-02-09/_temporary/_attempt_201503102148_0007_m_000017_0
+ //temp dir for this map task...
+ Path tempDir = new Path(tempLocation);
+ if(fs.exists(tempDir)){
+ boolean tempLocationDel = fs.delete(new Path(tempLocation), true);
+ if(!tempLocationDel){
+ // give second chance for the deleting the tempDir...by the hadoop framework if it failed......
+ fs.deleteOnExit(tempDir);
+ }
+ }
+ long duration = System.currentTimeMillis() - startTime;
+ log.info("Finish copy-rename-commit at " + new Date() + " and duration (ms) is " + duration );
+ CamusJob.stopTiming("copy-rename-commit");
+ }
+ }
+ else {
+ // This is default behavior out of box adding more
+ // should we retry if rename fails here and remove the data...
+ try{
+ boolean result = FileSystem.get(job.getConfiguration()).rename(source, target);
+ if(!result){
+ /*
+ * We do not want silent failure...: should we do the Md5 or CRC checksum ..?
+ * Based on feedback please refer to https://github.com/linkedin/camus/issues/189
+ * We do not throw exception data will be lost..
+ */
+ String msg = "Could not rename a File during Commit Phase due to underlying File System reutrn false result: soruce=" + source.toString() +" to target=" + target.toString();
+ log.error(msg);
+ throw new IOException(msg);
+ }
+ }catch(Throwable e){
+ log.error("File Rename Failed from= " + source.getName() +" to=" + target.getName() , e);
+ throw new RuntimeException("Default Behavior file commit failed due to fatal problem",e);
+ }
+ }
+ }
+
+ static class CopyToTempTask implements RetryLogic.Delegate<Boolean> {
+
+
+ private Path tempFilePath;
+ private String tempLocation;
+ private Path source;
+ private FileSystem fs;
+ // TODO Why logger is not static per class ... why instance variable ...?
+ private Logger log;
+
+
+ CopyToTempTask(String tempLocation, Path source, FileSystem fs, Logger log ){
+ this.tempLocation = tempLocation;
+ this.source = source;
+ this.fs = fs;
+ this.log = log;
+ }
+
+ @Override
+ public Boolean call() throws Exception {
+ Path destinationTemp = new Path(tempLocation + "/" + source.getName());
+ boolean result = false;
+ int retry = 3;
+ // for ANY fatal underlying FS error try 3 more times...
+ // try to create a temp directory try 3 times if there is any failure then do not try...
+ while(retry > 0 && !result){
+ Path origDestinationTemp = new Path(destinationTemp.toString());
+ try{
+ result = FileUtil.copy(fs, source, fs,
+ destinationTemp, false, fs.getConf());
+ tempFilePath = destinationTemp;
+
+ if(!result){
+ log.error("Failed rename to temp destination Path so delete it : " + destinationTemp.toString());
+ // if result is failure then lets delete it now...
+ boolean deleted = fs.delete(destinationTemp,true);
+ if(!deleted){
+ // let see if hadoop framework can delete this..otherwise this will be garbage sitting here for ever and ever...TODO ????
+ fs.deleteOnExit(destinationTemp);
+ // try new path...
+ destinationTemp = new Path(destinationTemp.toString() + "-"+retry);
+ }
+ }
+ }catch(IOException e){
+ log.error("Error while copy to temp location from=" + source.toString() +" and to=" + origDestinationTemp.toString(),e);
+ try{
+ // try to close file and delete it... TODO Review this....
+ fs.create(origDestinationTemp,true).close();
+ fs.delete(origDestinationTemp, true);
+ }catch(Throwable th){
+ log.error("Error delete while copy to temp location from=" + source.toString() +" and to=" + origDestinationTemp.toString(),e); }
+
+ }
+ }
+ return result;
+ }
+
+ public Path getTempPath(){
+ return tempFilePath;
+ }
+ }
public String getPartitionedPath(JobContext context, String file, int count, long offset) throws IOException {
Matcher m = workingFileMetadataPattern.matcher(file);
diff --git a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputFormat.java b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputFormat.java
index fda0250..e850779 100644
--- a/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputFormat.java
+++ b/camus-etl-kafka/src/main/java/com/linkedin/camus/etl/kafka/mapred/EtlMultiOutputFormat.java
@@ -48,6 +48,8 @@ public class EtlMultiOutputFormat extends FileOutputFormat<EtlKey, Object> {
public static final String ETL_OUTPUT_CODEC = "etl.output.codec";
public static final String ETL_DEFAULT_OUTPUT_CODEC = "deflate";
public static final String ETL_RECORD_WRITER_PROVIDER_CLASS = "etl.record.writer.provider.class";
+ public static final String ETL_RECORD_WRITER_TWO_STEPS_COPY_TO_TEMP_AND_RENAME = "etl.output.two.steps.commit.copyToTemp.and.rename";
+
public static final DateTimeFormatter FILE_DATE_FORMATTER = DateUtils.getDateTimeFormatter("YYYYMMddHH");
public static final String OFFSET_PREFIX = "offsets";
@@ -60,12 +62,31 @@ public class EtlMultiOutputFormat extends FileOutputFormat<EtlKey, Object> {
private static Logger log = Logger.getLogger(EtlMultiOutputFormat.class);
+
@Override
- public RecordWriter<EtlKey, Object> getRecordWriter(TaskAttemptContext context) throws IOException,
- InterruptedException {
- if (committer == null)
- committer = new EtlMultiOutputCommitter(getOutputPath(context), context, log);
- return new EtlMultiOutputRecordWriter(context, committer);
+ public RecordWriter<EtlKey, Object> getRecordWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+
+ /**
+ * This is double locking check but have to be done so we do not have two Committer may cause this error.
+ * http://en.wikipedia.org/wiki/Double-checked_locking but this should be fine
+ *
+ * http://www.javamex.com/tutorials/double_checked_locking_fixing.shtml
+ * Please refer see section "3. Use DCL plus volatile"
+ *
+ * I can not use the lazy or early load since I need "context" to instantiate object...
+ *
+ * We want to ensure that there is only one and only object for this ETL JOB...