/
StringRecordWriterProvider.java
139 lines (116 loc) · 5.08 KB
/
StringRecordWriterProvider.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
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.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;
/**
* Provides a RecordWriter that uses FSDataOutputStream to write
* a String record as bytes to HDFS without any reformatting or compression.
*/
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";
protected String recordDelimiter = null;
private String extension = "";
private boolean isCompressed = false;
private CompressionCodec codec = null;
public StringRecordWriterProvider(TaskAttemptContext context) {
Configuration conf = context.getConfiguration();
if (recordDelimiter == null) {
recordDelimiter = conf.get(ETL_OUTPUT_RECORD_DELIMITER, DEFAULT_RECORD_DELIMITER);
}
isCompressed = FileOutputFormat.getCompressOutput(context);
if (isCompressed) {
Class<? extends CompressionCodec> codecClass = null;
if ("snappy".equals(EtlMultiOutputFormat.getEtlOutputCodec(context))) {
codecClass = SnappyCodec.class;
} else if ("gzip".equals((EtlMultiOutputFormat.getEtlOutputCodec(context)))) {
codecClass = GzipCodec.class;
} else {
codecClass = DefaultCodec.class;
}
codec = ReflectionUtils.newInstance(codecClass, conf);
extension = codec.getDefaultExtension();
}
}
// TODO: Make this configurable somehow.
// To do this, we'd have to make RecordWriterProvider have an
// init(JobContext context) method signature that EtlMultiOutputFormat would always call.
@Override
public String getFilenameExtension() {
return extension;
}
@Override
public RecordWriter<IEtlKey, CamusWrapper> getDataRecordWriter(TaskAttemptContext context, String fileName,
CamusWrapper camusWrapper, FileOutputCommitter committer) throws IOException, InterruptedException {
// If recordDelimiter hasn't been initialized, do so now
if (recordDelimiter == null) {
recordDelimiter = context.getConfiguration().get(ETL_OUTPUT_RECORD_DELIMITER, DEFAULT_RECORD_DELIMITER);
}
// Get the filename for this RecordWriter.
Path path =
new Path(committer.getWorkPath(), EtlMultiOutputFormat.getUniqueFile(context, fileName, getFilenameExtension()));
FileSystem fs = path.getFileSystem(context.getConfiguration());
if (!isCompressed) {
FSDataOutputStream fileOut = fs.create(path, false);
return new ByteRecordWriter(fileOut, recordDelimiter);
} else {
FSDataOutputStream fileOut = fs.create(path, false);
return new ByteRecordWriter(new DataOutputStream(codec.createOutputStream(fileOut)), recordDelimiter);
}
/*
// Create a FSDataOutputStream stream that will write to path.
final FSDataOutputStream writer = path.getFileSystem(context.getConfiguration()).create(path);
// Return a new anonymous RecordWriter that uses the
// FSDataOutputStream writer to write bytes straight into path.
return new RecordWriter<IEtlKey, CamusWrapper>() {
@Override
public void write(IEtlKey ignore, CamusWrapper data) throws IOException {
String record = (String)data.getRecord() + recordDelimiter;
writer.write(record.getBytes());
}
@Override
public void close(TaskAttemptContext context) throws IOException, InterruptedException {
writer.close();
}
};
*/
}
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 close(TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
out.close();
}
}
}