/
RollingFileSink.java
292 lines (248 loc) · 9.28 KB
/
RollingFileSink.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
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flume.sink;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.flume.Channel;
import org.apache.flume.Context;
import org.apache.flume.Event;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.Transaction;
import org.apache.flume.conf.BatchSizeSupported;
import org.apache.flume.conf.Configurable;
import org.apache.flume.formatter.output.PathManager;
import org.apache.flume.formatter.output.PathManagerFactory;
import org.apache.flume.instrumentation.SinkCounter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.flume.serialization.EventSerializer;
import org.apache.flume.serialization.EventSerializerFactory;
public class RollingFileSink extends AbstractSink implements Configurable, BatchSizeSupported {
private static final Logger logger = LoggerFactory
.getLogger(RollingFileSink.class);
private static final long defaultRollInterval = 30;
private static final int defaultBatchSize = 100;
private int batchSize = defaultBatchSize;
private File directory;
private long rollInterval;
private OutputStream outputStream;
private ScheduledExecutorService rollService;
private String serializerType;
private Context serializerContext;
private EventSerializer serializer;
private SinkCounter sinkCounter;
private PathManager pathController;
private volatile boolean shouldRotate;
public RollingFileSink() {
shouldRotate = false;
}
@Override
public void configure(Context context) {
String pathManagerType = context.getString("sink.pathManager", "DEFAULT");
String directory = context.getString("sink.directory");
String rollInterval = context.getString("sink.rollInterval");
serializerType = context.getString("sink.serializer", "TEXT");
serializerContext =
new Context(context.getSubProperties("sink." +
EventSerializer.CTX_PREFIX));
Context pathManagerContext =
new Context(context.getSubProperties("sink." +
PathManager.CTX_PREFIX));
pathController = PathManagerFactory.getInstance(pathManagerType, pathManagerContext);
Preconditions.checkArgument(directory != null, "Directory may not be null");
Preconditions.checkNotNull(serializerType, "Serializer type is undefined");
if (rollInterval == null) {
this.rollInterval = defaultRollInterval;
} else {
this.rollInterval = Long.parseLong(rollInterval);
}
batchSize = context.getInteger("sink.batchSize", defaultBatchSize);
this.directory = new File(directory);
if (sinkCounter == null) {
sinkCounter = new SinkCounter(getName());
}
}
@Override
public void start() {
logger.info("Starting {}...", this);
sinkCounter.start();
super.start();
pathController.setBaseDirectory(directory);
if (rollInterval > 0) {
rollService = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder().setNameFormat(
"rollingFileSink-roller-" +
Thread.currentThread().getId() + "-%d").build());
/*
* Every N seconds, mark that it's time to rotate. We purposefully do NOT
* touch anything other than the indicator flag to avoid error handling
* issues (e.g. IO exceptions occuring in two different threads.
* Resist the urge to actually perform rotation in a separate thread!
*/
rollService.scheduleAtFixedRate(new Runnable() {
@Override
public void run() {
logger.debug("Marking time to rotate file {}",
pathController.getCurrentFile());
shouldRotate = true;
}
}, rollInterval, rollInterval, TimeUnit.SECONDS);
} else {
logger.info("RollInterval is not valid, file rolling will not happen.");
}
logger.info("RollingFileSink {} started.", getName());
}
@Override
public Status process() throws EventDeliveryException {
if (shouldRotate) {
logger.debug("Time to rotate {}", pathController.getCurrentFile());
if (outputStream != null) {
logger.debug("Closing file {}", pathController.getCurrentFile());
try {
serializer.flush();
serializer.beforeClose();
outputStream.close();
sinkCounter.incrementConnectionClosedCount();
shouldRotate = false;
} catch (IOException e) {
sinkCounter.incrementConnectionFailedCount();
throw new EventDeliveryException("Unable to rotate file "
+ pathController.getCurrentFile() + " while delivering event", e);
} finally {
serializer = null;
outputStream = null;
}
pathController.rotate();
}
}
if (outputStream == null) {
File currentFile = pathController.getCurrentFile();
logger.debug("Opening output stream for file {}", currentFile);
try {
outputStream = new BufferedOutputStream(
new FileOutputStream(currentFile));
serializer = EventSerializerFactory.getInstance(
serializerType, serializerContext, outputStream);
serializer.afterCreate();
sinkCounter.incrementConnectionCreatedCount();
} catch (IOException e) {
sinkCounter.incrementConnectionFailedCount();
throw new EventDeliveryException("Failed to open file "
+ pathController.getCurrentFile() + " while delivering event", e);
}
}
Channel channel = getChannel();
Transaction transaction = channel.getTransaction();
Event event = null;
Status result = Status.READY;
try {
transaction.begin();
int eventAttemptCounter = 0;
for (int i = 0; i < batchSize; i++) {
event = channel.take();
if (event != null) {
sinkCounter.incrementEventDrainAttemptCount();
eventAttemptCounter++;
serializer.write(event);
/*
* FIXME: Feature: Rotate on size and time by checking bytes written and
* setting shouldRotate = true if we're past a threshold.
*/
/*
* FIXME: Feature: Control flush interval based on time or number of
* events. For now, we're super-conservative and flush on each write.
*/
} else {
// No events found, request back-off semantics from runner
result = Status.BACKOFF;
break;
}
}
serializer.flush();
outputStream.flush();
transaction.commit();
sinkCounter.addToEventDrainSuccessCount(eventAttemptCounter);
} catch (Exception ex) {
sinkCounter.incrementEventWriteOrChannelFail(ex);
transaction.rollback();
throw new EventDeliveryException("Failed to process transaction", ex);
} finally {
transaction.close();
}
return result;
}
@Override
public void stop() {
logger.info("RollingFile sink {} stopping...", getName());
sinkCounter.stop();
super.stop();
if (outputStream != null) {
logger.debug("Closing file {}", pathController.getCurrentFile());
try {
serializer.flush();
serializer.beforeClose();
outputStream.close();
sinkCounter.incrementConnectionClosedCount();
} catch (IOException e) {
sinkCounter.incrementConnectionFailedCount();
logger.error("Unable to close output stream. Exception follows.", e);
} finally {
outputStream = null;
serializer = null;
}
}
if (rollInterval > 0) {
rollService.shutdown();
while (!rollService.isTerminated()) {
try {
rollService.awaitTermination(1, TimeUnit.SECONDS);
} catch (InterruptedException e) {
logger.debug("Interrupted while waiting for roll service to stop. " +
"Please report this.", e);
}
}
}
logger.info("RollingFile sink {} stopped. Event metrics: {}",
getName(), sinkCounter);
}
public File getDirectory() {
return directory;
}
public void setDirectory(File directory) {
this.directory = directory;
}
public long getRollInterval() {
return rollInterval;
}
public void setRollInterval(long rollInterval) {
this.rollInterval = rollInterval;
}
@Override
public long getBatchSize() {
return batchSize;
}
}