forked from apache/incubator-nemo
/
SourceVertexDataFetcher.java
115 lines (103 loc) · 3.65 KB
/
SourceVertexDataFetcher.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
/*
* 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.nemo.runtime.executor.task;
import org.apache.nemo.common.ir.OutputCollector;
import org.apache.nemo.common.ir.Readable;
import org.apache.nemo.common.ir.vertex.SourceVertex;
import org.apache.nemo.common.punctuation.Finishmark;
import org.apache.nemo.common.punctuation.Watermark;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.NoSuchElementException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
/**
* Fetches data from a data source.
*/
class SourceVertexDataFetcher extends DataFetcher {
private static final Logger LOG = LoggerFactory.getLogger(SourceVertexDataFetcher.class.getName());
private final Readable readable;
private long boundedSourceReadTime = 0;
private static final long WATERMARK_PERIOD = 1000; // ms
private final ScheduledExecutorService watermarkTriggerService;
private boolean watermarkTriggered = false;
private final boolean bounded;
SourceVertexDataFetcher(final SourceVertex dataSource,
final Readable readable,
final OutputCollector outputCollector) {
super(dataSource, outputCollector);
this.readable = readable;
this.readable.prepare();
this.bounded = dataSource.isBounded();
if (!bounded) {
this.watermarkTriggerService = Executors.newScheduledThreadPool(1);
this.watermarkTriggerService.scheduleAtFixedRate(() ->
watermarkTriggered = true
, WATERMARK_PERIOD, WATERMARK_PERIOD, TimeUnit.MILLISECONDS);
} else {
this.watermarkTriggerService = null;
}
}
/**
* This is non-blocking operation.
*
* @return current data
* @throws NoSuchElementException if the current data is not available
*/
@Override
Object fetchDataElement() throws NoSuchElementException, IOException {
if (readable.isFinished()) {
return Finishmark.getInstance();
} else {
final long start = System.currentTimeMillis();
final Object element = retrieveElement();
boundedSourceReadTime += System.currentTimeMillis() - start;
return element;
}
}
final long getBoundedSourceReadTime() {
return boundedSourceReadTime;
}
@Override
public void close() throws Exception {
readable.close();
if (watermarkTriggerService != null) {
watermarkTriggerService.shutdown();
}
}
private boolean isWatermarkTriggerTime() {
if (watermarkTriggered) {
watermarkTriggered = false;
return true;
} else {
return false;
}
}
private Object retrieveElement() throws NoSuchElementException, IOException {
// Emit watermark
if (!bounded && isWatermarkTriggerTime()) {
return new Watermark(readable.readWatermark());
}
// Data
final Object element = readable.readCurrent();
return element;
}
}