├── README.md
├── pom.xml
└── src
└── main
└── java
└── com
└── baidu
└── unbiz
└── flume
└── sink
├── DateUtil.java
├── InputNotSpecifiedException.java
├── OutputStreamWrapper.java
├── RollingByTypeAndDayFileSink.java
├── StaticLinePrefixExecSource.java
└── SystemPropertiesUtil.java
/README.md:
--------------------------------------------------------------------------------
1 | 辅助[Flume+Kafka收集分布式日志应用实践](http://neoremind.com/2016/05/Flume+Kafka%E6%94%B6%E9%9B%86Docker%E5%AE%B9%E5%99%A8%E5%86%85%E5%88%86%E5%B8%83%E5%BC%8F%E6%97%A5%E5%BF%97%E5%BA%94%E7%94%A8%E5%AE%9E%E8%B7%B5/)的自定义flume的source和sink
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
3 | 4.0.0
4 | com.baidu.unbiz
5 | flume-utilty
6 | 1.0.0-SNAPSHOT
7 | jar
8 | Flume NG kafka to file sink
9 |
10 | 1.7
11 | 1.7
12 | UTF-8
13 | 1.6.0
14 |
15 |
16 |
17 |
18 | org.apache.flume.flume-ng-sinks
19 | flume-ng-kafka-sink
20 | ${flume.version}
21 |
22 |
23 | org.apache.flume
24 | flume-ng-core
25 | ${flume.version}
26 |
27 |
28 |
29 | package
30 | ${project.artifactId}
31 | target/classes
32 |
33 |
34 |
35 | org.apache.maven.plugins
36 | maven-compiler-plugin
37 | 2.3.1
38 |
39 | ${source.version}
40 | ${target.version}
41 | ${encoding}
42 | true
43 | true
44 | true
45 |
46 |
47 | ${java.home}/lib/rt.jar
48 |
49 |
50 |
51 |
52 |
53 | org.apache.maven.plugins
54 | maven-source-plugin
55 | 2.3
56 |
57 |
58 | attach-sources
59 |
60 | jar
61 |
62 |
63 |
64 |
65 |
66 |
67 |
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/DateUtil.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | import java.text.SimpleDateFormat;
4 | import java.util.Calendar;
5 | import java.util.Date;
6 |
7 | import org.apache.commons.lang.StringUtils;
8 |
9 | /**
10 | * @author zhangxu
11 | */
12 | public class DateUtil {
13 |
14 | public static boolean isSameDay(Date date1, Date date2) {
15 | if (date1 != null && date2 != null) {
16 | Calendar cal1 = Calendar.getInstance();
17 | cal1.setTime(date1);
18 | Calendar cal2 = Calendar.getInstance();
19 | cal2.setTime(date2);
20 | return isSameDay(cal1, cal2);
21 | } else {
22 | throw new IllegalArgumentException("The date must not be null");
23 | }
24 | }
25 |
26 | public static boolean isSameDay(Calendar cal1, Calendar cal2) {
27 | if (cal1 != null && cal2 != null) {
28 | return cal1.get(0) == cal2.get(0) && cal1.get(1) == cal2.get(1) && cal1.get(6) == cal2.get(6);
29 | } else {
30 | throw new IllegalArgumentException("The date must not be null");
31 | }
32 | }
33 |
34 | public static String formatDate(Date date) {
35 | return formatDate(date, "yyyyMMdd");
36 | }
37 |
38 | public static String formatDate(Date date, String format) {
39 | return date != null && !StringUtils.isBlank(format) ? (new SimpleDateFormat(format)).format(date) : null;
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/InputNotSpecifiedException.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | /**
4 | * @author zhangxu
5 | */
6 | public class InputNotSpecifiedException extends RuntimeException {
7 | private static final long serialVersionUID = 1102327497549834945L;
8 |
9 | public InputNotSpecifiedException() {
10 | }
11 |
12 | public InputNotSpecifiedException(String message) {
13 | super(message);
14 | }
15 |
16 | public InputNotSpecifiedException(String message, Throwable t) {
17 | super(message, t);
18 | }
19 |
20 | public InputNotSpecifiedException(Throwable t) {
21 | super(t);
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/OutputStreamWrapper.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | import java.io.OutputStream;
4 | import java.util.Date;
5 |
6 | import org.apache.flume.serialization.EventSerializer;
7 |
8 | /**
9 | * @author zhangxu
10 | */
11 | public class OutputStreamWrapper {
12 |
13 | private OutputStream outputStream;
14 |
15 | private Date date;
16 |
17 | private EventSerializer serializer;
18 |
19 | public OutputStream getOutputStream() {
20 | return outputStream;
21 | }
22 |
23 | public void setOutputStream(OutputStream outputStream) {
24 | this.outputStream = outputStream;
25 | }
26 |
27 | public Date getDate() {
28 | return date;
29 | }
30 |
31 | public void setDate(Date date) {
32 | this.date = date;
33 | }
34 |
35 | public EventSerializer getSerializer() {
36 | return serializer;
37 | }
38 |
39 | public void setSerializer(EventSerializer serializer) {
40 | this.serializer = serializer;
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/RollingByTypeAndDayFileSink.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | import java.io.BufferedOutputStream;
4 | import java.io.File;
5 | import java.io.FileOutputStream;
6 | import java.io.IOException;
7 | import java.io.OutputStream;
8 | import java.io.UnsupportedEncodingException;
9 | import java.util.Calendar;
10 | import java.util.Collection;
11 | import java.util.Date;
12 | import java.util.Map;
13 | import java.util.concurrent.ConcurrentMap;
14 |
15 | import org.apache.commons.lang.StringUtils;
16 | import org.apache.flume.Channel;
17 | import org.apache.flume.Context;
18 | import org.apache.flume.Event;
19 | import org.apache.flume.EventDeliveryException;
20 | import org.apache.flume.Transaction;
21 | import org.apache.flume.conf.Configurable;
22 | import org.apache.flume.instrumentation.SinkCounter;
23 | import org.apache.flume.serialization.EventSerializer;
24 | import org.apache.flume.serialization.EventSerializerFactory;
25 | import org.apache.flume.sink.AbstractSink;
26 | import org.slf4j.Logger;
27 | import org.slf4j.LoggerFactory;
28 |
29 | import com.google.common.base.Preconditions;
30 | import com.google.common.collect.Maps;
31 |
32 | /**
33 | * 把Event
写入文件的sink
34 | *
35 | * 这里有两个条件:
36 | * 1)Event header中必须有timestamp,否则会忽略事件,并且会抛出{@link InputNotSpecifiedException}
37 | * 2)Event body如果是按照##$$##
分隔的,那么把分隔之前的字符串当做模块名称(module name)来处理;如果没有则默认为default文件名
38 | *
39 | * 输出到本地文件,首先要设置一个跟目录,通过sink.directory
设置。
40 | * 其次根据条件#2中提取出来的module name作为文件名称前缀,timestamp日志作为文件名称后缀,例如文件名为portal.20150606或者default.20150703。
41 | *
42 | * NOTE:具有rolling by day的功能,文件会按照timestamp进行天级别粒度的存储。
43 | *
44 | * @author zhangxu
45 | */
46 | public class RollingByTypeAndDayFileSink extends AbstractSink implements Configurable {
47 |
48 | private static final Logger logger = LoggerFactory.getLogger(RollingByTypeAndDayFileSink.class);
49 |
50 | private static final int defaultBatchSize = 100;
51 |
52 | private int batchSize = defaultBatchSize;
53 |
54 | private String directory;
55 | private ConcurrentMap fileName2OutputStream = Maps.newConcurrentMap();
56 |
57 | private String serializerType;
58 | private Context serializerContext;
59 |
60 | private SinkCounter sinkCounter;
61 |
62 | public RollingByTypeAndDayFileSink() {
63 | }
64 |
65 | @Override
66 | public void configure(Context context) {
67 | String directory = context.getString("sink.directory");
68 |
69 | serializerType = context.getString("sink.serializer", "TEXT");
70 | serializerContext =
71 | new Context(context.getSubProperties("sink." +
72 | EventSerializer.CTX_PREFIX));
73 |
74 | Preconditions.checkArgument(directory != null, "Directory may not be null");
75 | Preconditions.checkNotNull(serializerType, "Serializer type is undefined");
76 |
77 | batchSize = context.getInteger("sink.batchSize", defaultBatchSize);
78 |
79 | this.directory = directory;
80 |
81 | if (sinkCounter == null) {
82 | sinkCounter = new SinkCounter(getName());
83 | }
84 | }
85 |
86 | @Override
87 | public void start() {
88 | logger.info("Starting {}...", this);
89 | sinkCounter.start();
90 | super.start();
91 | logger.info("RollingByTypeAndDaySink {} started.", getName());
92 | }
93 |
94 | @Override
95 | public Status process() throws EventDeliveryException {
96 | Channel channel = getChannel();
97 | Transaction transaction = channel.getTransaction();
98 | Event event = null;
99 | Status result = Status.READY;
100 |
101 | try {
102 | transaction.begin();
103 | int eventAttemptCounter = 0;
104 | OutputStreamWrapper outputStreamWrapper = null;
105 | for (int i = 0; i < batchSize; i++) {
106 | event = channel.take();
107 | if (event != null) {
108 | sinkCounter.incrementEventDrainAttemptCount();
109 | eventAttemptCounter++;
110 |
111 | String moduleName = getModuleName(event);
112 |
113 | Date date = getDate(event.getHeaders());
114 | outputStreamWrapper = fileName2OutputStream.get(moduleName);
115 | if (outputStreamWrapper == null) {
116 | outputStreamWrapper = createOutputStreamWrapper(moduleName, date);
117 | fileName2OutputStream.put(moduleName, outputStreamWrapper);
118 | } else {
119 | if (!DateUtil.isSameDay(outputStreamWrapper.getDate(), date)) {
120 | logger.debug("Time to rotate {}", getFileByModuleName(moduleName, date));
121 | destroyOutputStreamWrapper(outputStreamWrapper, moduleName, date);
122 | outputStreamWrapper = createOutputStreamWrapper(moduleName, date);
123 | fileName2OutputStream.put(moduleName, outputStreamWrapper);
124 | }
125 | }
126 |
127 | outputStreamWrapper.getSerializer().write(event);
128 | } else {
129 | // No events found, request back-off semantics from runner
130 | result = Status.BACKOFF;
131 | break;
132 | }
133 | }
134 | if (outputStreamWrapper != null) {
135 | outputStreamWrapper.getSerializer().flush();
136 | outputStreamWrapper.getOutputStream().flush();
137 | }
138 | transaction.commit();
139 | sinkCounter.addToEventDrainSuccessCount(eventAttemptCounter);
140 | } catch (InputNotSpecifiedException ex) {
141 | transaction.rollback();
142 | //logger.error(ex.getMessage());
143 | } catch (Exception ex) {
144 | transaction.rollback();
145 | throw new EventDeliveryException("Failed to process transaction", ex);
146 | } finally {
147 | transaction.close();
148 | }
149 |
150 | return result;
151 | }
152 |
153 | @Override
154 | public void stop() {
155 | logger.info("RollingByTypeAndDay sink {} stopping...", getName());
156 | sinkCounter.stop();
157 | super.stop();
158 | Collection outputStreamWrapperCollection = fileName2OutputStream.values();
159 |
160 | if (outputStreamWrapperCollection != null) {
161 | for (OutputStreamWrapper outputStreamWrapper : outputStreamWrapperCollection) {
162 | destroyOutputStreamWrapper(outputStreamWrapper);
163 | }
164 | }
165 | logger.info("RollingByTypeAndDay sink {} stopped. Event metrics: {}", getName(), sinkCounter);
166 | }
167 |
168 | private String getModuleName(Event event) {
169 | try {
170 | String line = new String(event.getBody(), "UTF-8");
171 | String[] seps = line.split("##\\$\\$##");
172 | if (seps != null && seps.length == 2) {
173 | if (StringUtils.isNotEmpty(seps[1])) {
174 | event.setBody(seps[1].getBytes("UTF-8"));
175 | }
176 | return seps[0];
177 | }
178 | } catch (UnsupportedEncodingException e) {
179 | logger.error(e.getMessage());
180 | }
181 | return "default";
182 | }
183 |
184 | private Date getDate(Map eventHeaders) {
185 | String timestamp = eventHeaders.get("timestamp");
186 | if (StringUtils.isEmpty(timestamp)) {
187 | throw new InputNotSpecifiedException("timestamp cannot be found in event header");
188 | }
189 | long millis = 0L;
190 | if (!StringUtils.isEmpty(timestamp)) {
191 | try {
192 | millis = Long.parseLong(timestamp);
193 | } catch (Exception e) {
194 | throw new InputNotSpecifiedException("timestamp cannot be parsed in event header");
195 | }
196 | }
197 | Calendar calendar = Calendar.getInstance();
198 | calendar.setTimeInMillis(millis);
199 | return calendar.getTime();
200 | }
201 |
202 | public OutputStreamWrapper createOutputStreamWrapper(String moduleName, Date date) throws EventDeliveryException {
203 | OutputStreamWrapper outputStreamWrapper = new OutputStreamWrapper();
204 | File currentFile = getFileByModuleName(moduleName, date);
205 | logger.debug("Opening output stream for file {}", currentFile);
206 | try {
207 | OutputStream outputStream = new BufferedOutputStream(
208 | new FileOutputStream(currentFile));
209 | EventSerializer serializer = EventSerializerFactory.getInstance(
210 | serializerType, serializerContext, outputStream);
211 | serializer.afterCreate();
212 | outputStreamWrapper.setOutputStream(outputStream);
213 | outputStreamWrapper.setSerializer(serializer);
214 | outputStreamWrapper.setDate(date);
215 | sinkCounter.incrementConnectionCreatedCount();
216 | } catch (IOException e) {
217 | sinkCounter.incrementConnectionFailedCount();
218 | throw new EventDeliveryException("Failed to open file "
219 | + getFileByModuleName(moduleName, date) + " while delivering event", e);
220 | }
221 | return outputStreamWrapper;
222 | }
223 |
224 | public void destroyOutputStreamWrapper(OutputStreamWrapper outputStreamWrapper) {
225 | try {
226 | destroyOutputStreamWrapper(outputStreamWrapper, "", new Date());
227 | } catch (EventDeliveryException e) {
228 | // omit
229 | }
230 | }
231 |
232 | public void destroyOutputStreamWrapper(OutputStreamWrapper outputStreamWrapper, String moduleName, Date date)
233 | throws EventDeliveryException {
234 | if (outputStreamWrapper.getOutputStream() != null) {
235 | logger.debug("Closing file {}", getFileByModuleName(moduleName, date));
236 |
237 | try {
238 | outputStreamWrapper.getSerializer().flush();
239 | outputStreamWrapper.getSerializer().beforeClose();
240 | outputStreamWrapper.getOutputStream().close();
241 | sinkCounter.incrementConnectionClosedCount();
242 | } catch (IOException e) {
243 | sinkCounter.incrementConnectionFailedCount();
244 | throw new EventDeliveryException("Unable to rotate file "
245 | + getFileByModuleName(moduleName, date) + " while delivering event", e);
246 | } finally {
247 | outputStreamWrapper.setOutputStream(null);
248 | outputStreamWrapper.setSerializer(null);
249 | }
250 | }
251 | outputStreamWrapper = null;
252 | }
253 |
254 | /**
255 | * 返回应该写入的文件句柄,为${module}.YYYYMMDD
256 | *
257 | * @param moduleName 模块名称
258 | * @param date 日期
259 | *
260 | * @return 文件
261 | */
262 | public File getFileByModuleName(String moduleName, Date date) {
263 | return new File(this.directory, moduleName + "." + DateUtil.formatDate(date));
264 | }
265 |
266 | public String getDirectory() {
267 | return directory;
268 | }
269 |
270 | public void setDirectory(String directory) {
271 | this.directory = directory;
272 | }
273 |
274 | }
275 |
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/StaticLinePrefixExecSource.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | import java.io.BufferedReader;
4 | import java.io.IOException;
5 | import java.io.InputStreamReader;
6 | import java.nio.charset.Charset;
7 | import java.util.ArrayList;
8 | import java.util.List;
9 | import java.util.concurrent.ExecutorService;
10 | import java.util.concurrent.Executors;
11 | import java.util.concurrent.Future;
12 | import java.util.concurrent.ScheduledExecutorService;
13 | import java.util.concurrent.ScheduledFuture;
14 | import java.util.concurrent.TimeUnit;
15 |
16 | import org.apache.commons.lang.StringUtils;
17 | import org.apache.flume.Context;
18 | import org.apache.flume.Event;
19 | import org.apache.flume.EventDrivenSource;
20 | import org.apache.flume.SystemClock;
21 | import org.apache.flume.channel.ChannelProcessor;
22 | import org.apache.flume.conf.Configurable;
23 | import org.apache.flume.event.EventBuilder;
24 | import org.apache.flume.instrumentation.SourceCounter;
25 | import org.apache.flume.source.AbstractSource;
26 | import org.apache.flume.source.ExecSourceConfigurationConstants;
27 | import org.slf4j.Logger;
28 | import org.slf4j.LoggerFactory;
29 |
30 | import com.google.common.base.Preconditions;
31 | import com.google.common.util.concurrent.ThreadFactoryBuilder;
32 |
33 | /**
34 | * 带有默认前缀字符串加入ExecSource输出结果的{@link EventDrivenSource}
35 | *
36 | * 配合tail -F 执行的结果加入固定的前缀,也就是放到event body中。前缀不通过static interceptor放入event header。
37 | *
38 | * 例如将输出加入portal##$$##10.18.24.33这个默认前缀的配置方法如下:
39 | *
40 | * a1.sources.r1.type = com.baidu.unbiz.flume.sink.PrefixExecSource
41 | * a1.sources.r1.command = tail -F /home/work/zhangxu/flume/logback.log
42 | * a1.sources.r1.channels = c1
43 | * a1.sources.r1.prefix=portal
44 | * a1.sources.r1.separator=##$$##
45 | * a1.sources.r1.suffix=10.18.24.33
46 | *
47 | * 原始文件某行为
48 | * abcdefg,那么日志的输出会变成
49 | * portal##$$##10.18.24.33 \t abcdefg
50 | *
51 | * 另外,有prefix,suffix才生效,否则suffix不会单独存在。
52 | *
53 | * 最高优先级的配置方式是通过JVM参数加入-Dflume_execsource_prefix、-Dflume_execsource_separator和-Dflume_execsource_suffix,
54 | * 其次是环境变量中的flume_execsource_prefix、flume_execsource_separator和flume_execsource_suffix
55 | *
56 | * Note:由于flume的{@link org.apache.flume.source.ExecSource}没有hook的好方式,只能copy 1.0.6.的源代码。
57 | *
58 | * @author zhangxu
59 | */
60 | public class StaticLinePrefixExecSource extends AbstractSource implements EventDrivenSource, Configurable {
61 |
62 | private static final Logger logger = LoggerFactory.getLogger(StaticLinePrefixExecSource.class);
63 |
64 | private String shell;
65 | private String command;
66 | private SourceCounter sourceCounter;
67 | private ExecutorService executor;
68 | private Future> runnerFuture;
69 | private long restartThrottle;
70 | private boolean restart;
71 | private boolean logStderr;
72 | private Integer bufferCount;
73 | private long batchTimeout;
74 | private ExecRunnable runner;
75 | private Charset charset;
76 |
77 | /**
78 | * 前缀
79 | */
80 | private String prefix;
81 |
82 | /**
83 | * 前缀和后缀之间的分隔符
84 | */
85 | private String separator;
86 |
87 | /**
88 | * 后缀
89 | */
90 | private String suffix;
91 |
92 | /**
93 | * {@link #prefix} + {@link #separator} + {@link #suffix}的结果
94 | */
95 | private String linePrefix;
96 |
97 | @Override
98 | public void start() {
99 | logger.info("Exec source starting with command:{}", command);
100 |
101 | executor = Executors.newSingleThreadExecutor();
102 |
103 | runner = new ExecRunnable(shell, command, getChannelProcessor(), sourceCounter,
104 | restart, restartThrottle, logStderr, bufferCount, batchTimeout, charset, linePrefix);
105 |
106 | // FIXME: Use a callback-like executor / future to signal us upon failure.
107 | runnerFuture = executor.submit(runner);
108 |
109 | /*
110 | * NB: This comes at the end rather than the beginning of the method because
111 | * it sets our state to running. We want to make sure the executor is alive
112 | * and well first.
113 | */
114 | sourceCounter.start();
115 | super.start();
116 |
117 | logger.debug("Exec source started");
118 | }
119 |
120 | @Override
121 | public void stop() {
122 | logger.info("Stopping exec source with command:{}", command);
123 | if (runner != null) {
124 | runner.setRestart(false);
125 | runner.kill();
126 | }
127 |
128 | if (runnerFuture != null) {
129 | logger.debug("Stopping exec runner");
130 | runnerFuture.cancel(true);
131 | logger.debug("Exec runner stopped");
132 | }
133 | executor.shutdown();
134 |
135 | while (!executor.isTerminated()) {
136 | logger.debug("Waiting for exec executor service to stop");
137 | try {
138 | executor.awaitTermination(500, TimeUnit.MILLISECONDS);
139 | } catch (InterruptedException e) {
140 | logger.debug("Interrupted while waiting for exec executor service "
141 | + "to stop. Just exiting.");
142 | Thread.currentThread().interrupt();
143 | }
144 | }
145 |
146 | sourceCounter.stop();
147 | super.stop();
148 |
149 | logger.debug("Exec source with command:{} stopped. Metrics:{}", command,
150 | sourceCounter);
151 | }
152 |
153 | @Override
154 | public void configure(Context context) {
155 | command = context.getString("command");
156 |
157 | Preconditions.checkState(command != null,
158 | "The parameter command must be specified");
159 |
160 | restartThrottle = context.getLong(ExecSourceConfigurationConstants.CONFIG_RESTART_THROTTLE,
161 | ExecSourceConfigurationConstants.DEFAULT_RESTART_THROTTLE);
162 |
163 | restart = context.getBoolean(ExecSourceConfigurationConstants.CONFIG_RESTART,
164 | ExecSourceConfigurationConstants.DEFAULT_RESTART);
165 |
166 | logStderr = context.getBoolean(ExecSourceConfigurationConstants.CONFIG_LOG_STDERR,
167 | ExecSourceConfigurationConstants.DEFAULT_LOG_STDERR);
168 |
169 | bufferCount = context.getInteger(ExecSourceConfigurationConstants.CONFIG_BATCH_SIZE,
170 | ExecSourceConfigurationConstants.DEFAULT_BATCH_SIZE);
171 |
172 | batchTimeout = context.getLong(ExecSourceConfigurationConstants.CONFIG_BATCH_TIME_OUT,
173 | ExecSourceConfigurationConstants.DEFAULT_BATCH_TIME_OUT);
174 |
175 | charset = Charset.forName(context.getString(ExecSourceConfigurationConstants.CHARSET,
176 | ExecSourceConfigurationConstants.DEFAULT_CHARSET));
177 |
178 | shell = context.getString(ExecSourceConfigurationConstants.CONFIG_SHELL, null);
179 | prefix = context.getString("prefix", null);
180 | separator = context.getString("separator", null);
181 | suffix = context.getString("suffix", null);
182 |
183 | prefix = SystemPropertiesUtil.getSystemProperty("flume_execsource_prefix", "flume_execsource_prefix", prefix);
184 | separator =
185 | SystemPropertiesUtil.getSystemProperty("flume_execsource_separator", "flume_execsource_separator",
186 | separator);
187 | suffix = SystemPropertiesUtil.getSystemProperty("flume_execsource_suffix", "flume_execsource_suffix", suffix);
188 |
189 | // prefix和separator必须有,然后如果有suffix则加上
190 | // 如果有suffix,默认后面加入\t分隔
191 | if (StringUtils.isNotEmpty(prefix) && StringUtils.isNotEmpty(separator)) {
192 | linePrefix = prefix + separator;
193 | if (StringUtils.isNotEmpty(suffix)) {
194 | linePrefix = linePrefix + suffix + "\t";
195 | }
196 | }
197 |
198 | if (sourceCounter == null) {
199 | sourceCounter = new SourceCounter(getName());
200 | }
201 | }
202 |
203 | private static class ExecRunnable implements Runnable {
204 |
205 | public ExecRunnable(String shell, String command, ChannelProcessor channelProcessor,
206 | SourceCounter sourceCounter, boolean restart, long restartThrottle,
207 | boolean logStderr, int bufferCount, long batchTimeout, Charset charset,
208 | String linePrefix) {
209 | this.command = command;
210 | this.channelProcessor = channelProcessor;
211 | this.sourceCounter = sourceCounter;
212 | this.restartThrottle = restartThrottle;
213 | this.bufferCount = bufferCount;
214 | this.batchTimeout = batchTimeout;
215 | this.restart = restart;
216 | this.logStderr = logStderr;
217 | this.charset = charset;
218 | this.shell = shell;
219 | this.linePrefix = linePrefix;
220 | }
221 |
222 | private final String shell;
223 | private final String command;
224 | private final ChannelProcessor channelProcessor;
225 | private final SourceCounter sourceCounter;
226 | private volatile boolean restart;
227 | private final long restartThrottle;
228 | private final int bufferCount;
229 | private long batchTimeout;
230 | private final boolean logStderr;
231 | private final Charset charset;
232 | private Process process = null;
233 | private SystemClock systemClock = new SystemClock();
234 | private Long lastPushToChannel = systemClock.currentTimeMillis();
235 | ScheduledExecutorService timedFlushService;
236 | ScheduledFuture> future;
237 |
238 | private String linePrefix;
239 |
240 | @Override
241 | public void run() {
242 | do {
243 | String exitCode = "unknown";
244 | BufferedReader reader = null;
245 | String line = null;
246 | final List eventList = new ArrayList();
247 |
248 | timedFlushService = Executors.newSingleThreadScheduledExecutor(
249 | new ThreadFactoryBuilder().setNameFormat(
250 | "timedFlushExecService" +
251 | Thread.currentThread().getId() + "-%d").build());
252 | try {
253 | if (shell != null) {
254 | String[] commandArgs = formulateShellCommand(shell, command);
255 | process = Runtime.getRuntime().exec(commandArgs);
256 | } else {
257 | String[] commandArgs = command.split("\\s+");
258 | process = new ProcessBuilder(commandArgs).start();
259 | }
260 | reader = new BufferedReader(
261 | new InputStreamReader(process.getInputStream(), charset));
262 |
263 | // StderrLogger dies as soon as the input stream is invalid
264 | StderrReader stderrReader = new StderrReader(new BufferedReader(
265 | new InputStreamReader(process.getErrorStream(), charset)), logStderr);
266 | stderrReader.setName("StderrReader-[" + command + "]");
267 | stderrReader.setDaemon(true);
268 | stderrReader.start();
269 |
270 | future = timedFlushService.scheduleWithFixedDelay(
271 | new Runnable() {
272 | @Override
273 | public void run() {
274 | try {
275 | synchronized(eventList) {
276 | if (!eventList.isEmpty()
277 | && timeout()) {
278 | flushEventBatch(eventList);
279 | }
280 | }
281 | } catch (Exception e) {
282 | logger.error(
283 | "Exception occured when "
284 | + "processing event"
285 | + " batch",
286 | e);
287 | if (e instanceof
288 | InterruptedException) {
289 | Thread.currentThread()
290 | .interrupt();
291 | }
292 | }
293 | }
294 | },
295 | batchTimeout, batchTimeout, TimeUnit.MILLISECONDS);
296 |
297 | while ((line = reader.readLine()) != null) {
298 | synchronized(eventList) {
299 | sourceCounter.incrementEventReceivedCount();
300 |
301 | // 如果linePrefix不为空,则加入到某行的前面
302 | if (linePrefix != null) {
303 | eventList.add(EventBuilder.withBody((linePrefix + line).getBytes(charset)));
304 | } else {
305 | eventList.add(EventBuilder.withBody(line.getBytes(charset)));
306 | }
307 | if (eventList.size() >= bufferCount || timeout()) {
308 | flushEventBatch(eventList);
309 | }
310 | }
311 | }
312 |
313 | synchronized(eventList) {
314 | if (!eventList.isEmpty()) {
315 | flushEventBatch(eventList);
316 | }
317 | }
318 | } catch (Exception e) {
319 | logger.error("Failed while running command: " + command, e);
320 | if (e instanceof InterruptedException) {
321 | Thread.currentThread().interrupt();
322 | }
323 | } finally {
324 | if (reader != null) {
325 | try {
326 | reader.close();
327 | } catch (IOException ex) {
328 | logger.error("Failed to close reader for exec source", ex);
329 | }
330 | }
331 | exitCode = String.valueOf(kill());
332 | }
333 | if (restart) {
334 | logger.info("Restarting in {}ms, exit code {}", restartThrottle,
335 | exitCode);
336 | try {
337 | Thread.sleep(restartThrottle);
338 | } catch (InterruptedException e) {
339 | Thread.currentThread().interrupt();
340 | }
341 | } else {
342 | logger.info("Command [" + command + "] exited with " + exitCode);
343 | }
344 | } while (restart);
345 | }
346 |
347 | private void flushEventBatch(List eventList) {
348 | channelProcessor.processEventBatch(eventList);
349 | sourceCounter.addToEventAcceptedCount(eventList.size());
350 | eventList.clear();
351 | lastPushToChannel = systemClock.currentTimeMillis();
352 | }
353 |
354 | private boolean timeout() {
355 | return (systemClock.currentTimeMillis() - lastPushToChannel) >= batchTimeout;
356 | }
357 |
358 | private static String[] formulateShellCommand(String shell, String command) {
359 | String[] shellArgs = shell.split("\\s+");
360 | String[] result = new String[shellArgs.length + 1];
361 | System.arraycopy(shellArgs, 0, result, 0, shellArgs.length);
362 | result[shellArgs.length] = command;
363 | return result;
364 | }
365 |
366 | public int kill() {
367 | if (process != null) {
368 | synchronized(process) {
369 | process.destroy();
370 |
371 | try {
372 | int exitValue = process.waitFor();
373 |
374 | // Stop the Thread that flushes periodically
375 | if (future != null) {
376 | future.cancel(true);
377 | }
378 |
379 | if (timedFlushService != null) {
380 | timedFlushService.shutdown();
381 | while (!timedFlushService.isTerminated()) {
382 | try {
383 | timedFlushService.awaitTermination(500, TimeUnit.MILLISECONDS);
384 | } catch (InterruptedException e) {
385 | logger.debug("Interrupted while waiting for exec executor service "
386 | + "to stop. Just exiting.");
387 | Thread.currentThread().interrupt();
388 | }
389 | }
390 | }
391 | return exitValue;
392 | } catch (InterruptedException ex) {
393 | Thread.currentThread().interrupt();
394 | }
395 | }
396 | return Integer.MIN_VALUE;
397 | }
398 | return Integer.MIN_VALUE / 2;
399 | }
400 |
401 | public void setRestart(boolean restart) {
402 | this.restart = restart;
403 | }
404 | }
405 |
406 | private static class StderrReader extends Thread {
407 | private BufferedReader input;
408 | private boolean logStderr;
409 |
410 | protected StderrReader(BufferedReader input, boolean logStderr) {
411 | this.input = input;
412 | this.logStderr = logStderr;
413 | }
414 |
415 | @Override
416 | public void run() {
417 | try {
418 | int i = 0;
419 | String line = null;
420 | while ((line = input.readLine()) != null) {
421 | if (logStderr) {
422 | // There is no need to read 'line' with a charset
423 | // as we do not to propagate it.
424 | // It is in UTF-16 and would be printed in UTF-8 format.
425 | logger.info("StderrLogger[{}] = '{}'", ++i, line);
426 | }
427 | }
428 | } catch (IOException e) {
429 | logger.info("StderrLogger exiting", e);
430 | } finally {
431 | try {
432 | if (input != null) {
433 | input.close();
434 | }
435 | } catch (IOException ex) {
436 | logger.error("Failed to close stderr reader for exec source", ex);
437 | }
438 | }
439 | }
440 | }
441 | }
--------------------------------------------------------------------------------
/src/main/java/com/baidu/unbiz/flume/sink/SystemPropertiesUtil.java:
--------------------------------------------------------------------------------
1 | package com.baidu.unbiz.flume.sink;
2 |
3 | import java.lang.management.ManagementFactory;
4 |
5 | public class SystemPropertiesUtil {
6 |
7 | public static String getSystemProperty(String key, String defautValue) {
8 | String value = System.getProperty(key);
9 | if (value == null || value.length() == 0) {
10 | value = System.getenv(key);
11 | if (value == null || value.length() == 0) {
12 | value = defautValue;
13 | }
14 | }
15 |
16 | return value;
17 | }
18 |
19 | public static String getSystemProperty(String dKey, String shellKey, String defautValue) {
20 | String value = System.getProperty(dKey);
21 | if (value == null || value.length() == 0) {
22 | value = System.getenv(shellKey);
23 | if (value == null || value.length() == 0) {
24 | value = defautValue;
25 | }
26 | }
27 |
28 | return value;
29 | }
30 |
31 | public static boolean isDebug() {
32 | return ManagementFactory.getRuntimeMXBean().getInputArguments().toString().indexOf("-agentlib:jdwp") > 0;
33 | }
34 | }
35 |
36 |
--------------------------------------------------------------------------------