/
InputProcessorStepImpl.java
241 lines (206 loc) · 7.92 KB
/
InputProcessorStepImpl.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
/*
* 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.carbondata.processing.loading.steps;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.carbondata.common.CarbonIterator;
import org.apache.carbondata.core.datastore.row.CarbonRow;
import org.apache.carbondata.core.util.CarbonProperties;
import org.apache.carbondata.core.util.CarbonThreadFactory;
import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
import org.apache.carbondata.processing.loading.DataField;
import org.apache.carbondata.processing.loading.parser.RowParser;
import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl;
import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
/**
* It reads data from record reader and sends data to next step.
*/
public class InputProcessorStepImpl extends AbstractDataLoadProcessorStep {
private RowParser rowParser;
private CarbonIterator<Object[]>[] inputIterators;
// cores used in SDK writer, set by the user
private short sdkWriterCores;
/**
* executor service to execute the query
*/
public ExecutorService executorService;
boolean isRawDataRequired = false;
public InputProcessorStepImpl(CarbonDataLoadConfiguration configuration,
CarbonIterator<Object[]>[] inputIterators) {
super(configuration, null);
this.inputIterators = inputIterators;
this.sdkWriterCores = configuration.getWritingCoresCount();
}
@Override public DataField[] getOutput() {
return configuration.getDataFields();
}
@Override public void initialize() throws IOException {
super.initialize();
rowParser = new RowParserImpl(getOutput(), configuration);
executorService = Executors.newCachedThreadPool(new CarbonThreadFactory(
"InputProcessorPool:" + configuration.getTableIdentifier().getCarbonTableIdentifier()
.getTableName(), true));
// if logger is enabled then raw data will be required.
this.isRawDataRequired = CarbonDataProcessorUtil.isRawDataRequired(configuration);
}
@Override public Iterator<CarbonRowBatch>[] execute() {
int batchSize = CarbonProperties.getInstance().getBatchSize();
List<CarbonIterator<Object[]>>[] readerIterators =
CarbonDataProcessorUtil.partitionInputReaderIterators(inputIterators, sdkWriterCores);
Iterator<CarbonRowBatch>[] outIterators = new Iterator[readerIterators.length];
for (int i = 0; i < outIterators.length; i++) {
outIterators[i] =
new InputProcessorIterator(readerIterators[i], rowParser, batchSize,
configuration.isPreFetch(), executorService, rowCounter, isRawDataRequired);
}
return outIterators;
}
@Override public void close() {
if (!closed) {
super.close();
if (null != executorService) {
executorService.shutdownNow();
}
for (CarbonIterator inputIterator : inputIterators) {
inputIterator.close();
}
}
}
@Override protected String getStepName() {
return "Input Processor";
}
/**
* This iterator wraps the list of iterators and it starts iterating the each
* iterator of the list one by one. It also parse the data while iterating it.
*/
public static class InputProcessorIterator extends CarbonIterator<CarbonRowBatch> {
private List<CarbonIterator<Object[]>> inputIterators;
private CarbonIterator<Object[]> currentIterator;
private int counter;
private int batchSize;
private RowParser rowParser;
private Future<CarbonRowBatch> future;
private ExecutorService executorService;
private boolean nextBatch;
private boolean firstTime;
private boolean preFetch;
private AtomicLong rowCounter;
private boolean isRawDataRequired = false;
public InputProcessorIterator(List<CarbonIterator<Object[]>> inputIterators,
RowParser rowParser, int batchSize, boolean preFetch, ExecutorService executorService,
AtomicLong rowCounter, boolean isRawDataRequired) {
this.inputIterators = inputIterators;
this.batchSize = batchSize;
this.rowParser = rowParser;
this.counter = 0;
// Get the first iterator from the list.
currentIterator = inputIterators.get(counter++);
this.executorService = executorService;
this.rowCounter = rowCounter;
this.preFetch = preFetch;
this.nextBatch = false;
this.firstTime = true;
this.isRawDataRequired = isRawDataRequired;
}
@Override
public boolean hasNext() {
return nextBatch || internalHasNext();
}
private boolean internalHasNext() {
if (firstTime) {
firstTime = false;
currentIterator.initialize();
}
boolean hasNext = currentIterator.hasNext();
// If iterator is finished then check for next iterator.
if (!hasNext) {
currentIterator.close();
// Check next iterator is available in the list.
if (counter < inputIterators.size()) {
// Get the next iterator from the list.
currentIterator = inputIterators.get(counter++);
// Initialize the new iterator
currentIterator.initialize();
hasNext = internalHasNext();
}
}
return hasNext;
}
@Override
public CarbonRowBatch next() {
if (preFetch) {
return getCarbonRowBatchWithPreFetch();
} else {
return getBatch();
}
}
private CarbonRowBatch getCarbonRowBatchWithPreFetch() {
CarbonRowBatch result = null;
if (future == null) {
future = getCarbonRowBatch();
}
try {
result = future.get();
} catch (InterruptedException e) {
throw new RuntimeException(e);
} catch (ExecutionException e) {
throw new RuntimeException(e);
}
nextBatch = false;
if (hasNext()) {
nextBatch = true;
future = getCarbonRowBatch();
}
return result;
}
private Future<CarbonRowBatch> getCarbonRowBatch() {
return executorService.submit(new Callable<CarbonRowBatch>() {
@Override public CarbonRowBatch call() throws Exception {
return getBatch();
}
});
}
private CarbonRowBatch getBatch() {
// Create batch and fill it.
CarbonRowBatch carbonRowBatch = new CarbonRowBatch(batchSize);
int count = 0;
if (isRawDataRequired) {
while (internalHasNext() && count < batchSize) {
Object[] rawRow = currentIterator.next();
carbonRowBatch.addRow(new CarbonRow(rowParser.parseRow(rawRow), rawRow));
count++;
}
} else {
while (internalHasNext() && count < batchSize) {
carbonRowBatch.addRow(new CarbonRow(rowParser.parseRow(currentIterator.next())));
count++;
}
}
rowCounter.getAndAdd(carbonRowBatch.getSize());
return carbonRowBatch;
}
}
}