-
Notifications
You must be signed in to change notification settings - Fork 13k
/
StreamFormatAdapter.java
301 lines (253 loc) · 11.5 KB
/
StreamFormatAdapter.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
/*
* 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.flink.connector.file.src.impl;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.InputStreamFSInputWrapper;
import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.compression.StandardDeCompressors;
import org.apache.flink.connector.file.src.reader.BulkFormat;
import org.apache.flink.connector.file.src.reader.StreamFormat;
import org.apache.flink.connector.file.src.util.CheckpointedPosition;
import org.apache.flink.connector.file.src.util.IteratorResultIterator;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.util.IOUtils;
import org.apache.flink.util.MathUtils;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import static org.apache.flink.connector.file.src.util.Utils.doWithCleanupOnException;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Adapter to turn a {@link StreamFormat} into a {@link BulkFormat}. */
@Internal
public final class StreamFormatAdapter<T> implements BulkFormat<T, FileSourceSplit> {
private static final long serialVersionUID = 1L;
private final StreamFormat<T> streamFormat;
public StreamFormatAdapter(StreamFormat<T> streamFormat) {
this.streamFormat = checkNotNull(streamFormat);
}
@Override
public BulkFormat.Reader<T> createReader(
final Configuration config, final FileSourceSplit split) throws IOException {
final TrackingFsDataInputStream trackingStream =
openStream(split.path(), config, split.offset());
final long splitEnd = split.offset() + split.length();
return doWithCleanupOnException(
trackingStream,
() -> {
final StreamFormat.Reader<T> streamReader =
streamFormat.createReader(
config,
trackingStream,
trackingStream.getFileLength(),
splitEnd);
return new Reader<>(
streamReader, trackingStream, CheckpointedPosition.NO_OFFSET, 0L);
});
}
@Override
public BulkFormat.Reader<T> restoreReader(
final Configuration config, final FileSourceSplit split) throws IOException {
assert split.getReaderPosition().isPresent();
final CheckpointedPosition checkpointedPosition = split.getReaderPosition().get();
final TrackingFsDataInputStream trackingStream =
openStream(split.path(), config, split.offset());
final long splitEnd = split.offset() + split.length();
return doWithCleanupOnException(
trackingStream,
() -> {
// if there never was a checkpointed offset, yet, we need to initialize the
// reader like a fresh reader.
// see the JavaDocs on StreamFormat.restoreReader() for details
final StreamFormat.Reader<T> streamReader =
checkpointedPosition.getOffset() == CheckpointedPosition.NO_OFFSET
? streamFormat.createReader(
config,
trackingStream,
trackingStream.getFileLength(),
splitEnd)
: streamFormat.restoreReader(
config,
trackingStream,
checkpointedPosition.getOffset(),
trackingStream.getFileLength(),
splitEnd);
// skip the records to skip, but make sure we close the reader if something goes
// wrong
doWithCleanupOnException(
streamReader,
() -> {
long toSkip = checkpointedPosition.getRecordsAfterOffset();
while (toSkip > 0 && streamReader.read() != null) {
toSkip--;
}
});
return new Reader<>(
streamReader,
trackingStream,
checkpointedPosition.getOffset(),
checkpointedPosition.getRecordsAfterOffset());
});
}
@Override
public boolean isSplittable() {
return streamFormat.isSplittable();
}
@Override
public TypeInformation<T> getProducedType() {
return streamFormat.getProducedType();
}
private static TrackingFsDataInputStream openStream(
final Path file, final Configuration config, final long seekPosition)
throws IOException {
final FileSystem fs = file.getFileSystem();
final long fileLength = fs.getFileStatus(file).getLen();
final int fetchSize =
MathUtils.checkedDownCast(config.get(StreamFormat.FETCH_IO_SIZE).getBytes());
if (fetchSize <= 0) {
throw new IllegalConfigurationException(
String.format(
"The fetch size (%s) must be > 0, but is %d",
StreamFormat.FETCH_IO_SIZE.key(), fetchSize));
}
final InflaterInputStreamFactory<?> deCompressor =
StandardDeCompressors.getDecompressorForFileName(file.getPath());
final FSDataInputStream inStream = fs.open(file);
return doWithCleanupOnException(
inStream,
() -> {
final FSDataInputStream in =
deCompressor == null
? inStream
: new InputStreamFSInputWrapper(deCompressor.create(inStream));
in.seek(seekPosition);
return new TrackingFsDataInputStream(in, fileLength, fetchSize);
});
}
// ----------------------------------------------------------------------------------
/** The reader adapter, from {@link StreamFormat.Reader} to {@link BulkFormat.Reader}. */
public static final class Reader<T> implements BulkFormat.Reader<T> {
private final StreamFormat.Reader<T> reader;
private final TrackingFsDataInputStream stream;
private long lastOffset;
private long lastRecordsAfterOffset;
Reader(
final StreamFormat.Reader<T> reader,
final TrackingFsDataInputStream stream,
final long initialOffset,
final long initialSkipCount) {
this.reader = checkNotNull(reader);
this.stream = checkNotNull(stream);
this.lastOffset = initialOffset;
this.lastRecordsAfterOffset = initialSkipCount;
}
@Nullable
@Override
public RecordIterator<T> readBatch() throws IOException {
updateCheckpointedPosition();
stream.newBatch();
final ArrayList<T> result = new ArrayList<>();
T next;
while (stream.hasRemainingInBatch() && (next = reader.read()) != null) {
result.add(next);
}
if (result.isEmpty()) {
return null;
}
final RecordIterator<T> iter =
new IteratorResultIterator<>(
result.iterator(), lastOffset, lastRecordsAfterOffset);
lastRecordsAfterOffset += result.size();
return iter;
}
@Override
public void close() throws IOException {
try {
reader.close();
} finally {
// this is just in case, to guard against resource leaks
IOUtils.closeQuietly(stream);
}
}
private void updateCheckpointedPosition() {
final CheckpointedPosition position = reader.getCheckpointedPosition();
if (position != null) {
this.lastOffset = position.getOffset();
this.lastRecordsAfterOffset = position.getRecordsAfterOffset();
}
}
}
// ----------------------------------------------------------------------------------
/**
* Utility stream that tracks how much has been read. This is used to decide when the reader
* should finish the current batch and start the next batch. That way we make the batch sizes
* dependent on the consumed data volume, which is more robust than making it dependent on a
* record count.
*/
private static final class TrackingFsDataInputStream extends FSDataInputStream {
private final FSDataInputStream stream;
private final long fileLength;
private final int batchSize;
private int remainingInBatch;
TrackingFsDataInputStream(FSDataInputStream stream, long fileLength, int batchSize) {
checkArgument(fileLength > 0L);
checkArgument(batchSize > 0);
this.stream = stream;
this.fileLength = fileLength;
this.batchSize = batchSize;
}
@Override
public void seek(long desired) throws IOException {
stream.seek(desired);
remainingInBatch = 0; // after each seek, we need to start a new batch
}
@Override
public long getPos() throws IOException {
return stream.getPos();
}
@Override
public int read() throws IOException {
remainingInBatch--;
return stream.read();
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
remainingInBatch -= len;
return stream.read(b, off, len);
}
@Override
public void close() throws IOException {
stream.close();
}
boolean hasRemainingInBatch() {
return remainingInBatch > 0;
}
void newBatch() {
remainingInBatch = batchSize;
}
long getFileLength() {
return fileLength;
}
}
}