-
Notifications
You must be signed in to change notification settings - Fork 5.4k
/
HiveSplitSource.java
545 lines (491 loc) · 23.1 KB
/
HiveSplitSource.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
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
/*
* Licensed 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 com.facebook.presto.hive;
import com.facebook.presto.hive.InternalHiveSplit.InternalHiveBlock;
import com.facebook.presto.hive.util.AsyncQueue;
import com.facebook.presto.hive.util.AsyncQueue.BorrowResult;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.connector.ConnectorPartitionHandle;
import com.facebook.presto.spi.predicate.TupleDomain;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.log.Logger;
import io.airlift.stats.CounterStat;
import io.airlift.units.DataSize;
import java.io.FileNotFoundException;
import java.util.List;
import java.util.Map;
import java.util.OptionalInt;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Predicate;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_FILE_NOT_FOUND;
import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR;
import static com.facebook.presto.hive.HiveSessionProperties.getMaxInitialSplitSize;
import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize;
import static com.facebook.presto.hive.HiveSplitSource.StateKind.CLOSED;
import static com.facebook.presto.hive.HiveSplitSource.StateKind.FAILED;
import static com.facebook.presto.hive.HiveSplitSource.StateKind.INITIAL;
import static com.facebook.presto.hive.HiveSplitSource.StateKind.NO_MORE_SPLITS;
import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.Maps.transformValues;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.failedFuture;
import static io.airlift.concurrent.MoreFutures.toCompletableFuture;
import static io.airlift.units.DataSize.succinctBytes;
import static java.lang.Math.min;
import static java.lang.Math.toIntExact;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
class HiveSplitSource
implements ConnectorSplitSource
{
private static final Logger log = Logger.get(HiveSplit.class);
private final String queryId;
private final String databaseName;
private final String tableName;
private final TupleDomain<? extends ColumnHandle> compactEffectivePredicate;
private final PerBucket queues;
private final AtomicInteger bufferedInternalSplitCount = new AtomicInteger();
private final int maxOutstandingSplitsBytes;
private final DataSize maxSplitSize;
private final DataSize maxInitialSplitSize;
private final AtomicInteger remainingInitialSplits;
private final HiveSplitLoader splitLoader;
private final AtomicReference<State> stateReference;
private final AtomicLong estimatedSplitSizeInBytes = new AtomicLong();
private final CounterStat highMemorySplitSourceCounter;
private final AtomicBoolean loggedHighMemoryWarning = new AtomicBoolean();
private HiveSplitSource(
ConnectorSession session,
String databaseName,
String tableName,
TupleDomain<? extends ColumnHandle> compactEffectivePredicate,
PerBucket queues,
int maxInitialSplits,
DataSize maxOutstandingSplitsSize,
HiveSplitLoader splitLoader,
AtomicReference<State> stateReference,
CounterStat highMemorySplitSourceCounter)
{
requireNonNull(session, "session is null");
this.queryId = session.getQueryId();
this.databaseName = requireNonNull(databaseName, "databaseName is null");
this.tableName = requireNonNull(tableName, "tableName is null");
this.compactEffectivePredicate = requireNonNull(compactEffectivePredicate, "compactEffectivePredicate is null");
this.queues = requireNonNull(queues, "queues is null");
this.maxOutstandingSplitsBytes = toIntExact(maxOutstandingSplitsSize.toBytes());
this.splitLoader = requireNonNull(splitLoader, "splitLoader is null");
this.stateReference = requireNonNull(stateReference, "stateReference is null");
this.highMemorySplitSourceCounter = requireNonNull(highMemorySplitSourceCounter, "highMemorySplitSourceCounter is null");
this.maxSplitSize = getMaxSplitSize(session);
this.maxInitialSplitSize = getMaxInitialSplitSize(session);
this.remainingInitialSplits = new AtomicInteger(maxInitialSplits);
}
public static HiveSplitSource allAtOnce(
ConnectorSession session,
String databaseName,
String tableName,
TupleDomain<? extends ColumnHandle> compactEffectivePredicate,
int maxInitialSplits,
int maxOutstandingSplits,
DataSize maxOutstandingSplitsSize,
HiveSplitLoader splitLoader,
Executor executor,
CounterStat highMemorySplitSourceCounter)
{
AtomicReference<State> stateReference = new AtomicReference<>(State.initial());
return new HiveSplitSource(
session,
databaseName,
tableName,
compactEffectivePredicate,
new PerBucket()
{
private final AsyncQueue<InternalHiveSplit> queue = new AsyncQueue<>(maxOutstandingSplits, executor);
@Override
public ListenableFuture<?> offer(OptionalInt bucketNumber, InternalHiveSplit connectorSplit)
{
// bucketNumber can be non-empty because BackgroundHiveSplitLoader does not have knowledge of execution plan
return queue.offer(connectorSplit);
}
@Override
public <O> ListenableFuture<O> borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function<List<InternalHiveSplit>, BorrowResult<InternalHiveSplit, O>> function)
{
checkArgument(!bucketNumber.isPresent());
return queue.borrowBatchAsync(maxSize, function);
}
@Override
public void finish()
{
queue.finish();
}
@Override
public boolean isFinished(OptionalInt bucketNumber)
{
checkArgument(!bucketNumber.isPresent());
return queue.isFinished();
}
},
maxInitialSplits,
maxOutstandingSplitsSize,
splitLoader,
stateReference,
highMemorySplitSourceCounter);
}
public static HiveSplitSource bucketed(
ConnectorSession session,
String databaseName,
String tableName,
TupleDomain<? extends ColumnHandle> compactEffectivePredicate,
int estimatedOutstandingSplitsPerBucket,
int maxInitialSplits,
DataSize maxOutstandingSplitsSize,
HiveSplitLoader splitLoader,
Executor executor,
CounterStat highMemorySplitSourceCounter)
{
AtomicReference<State> stateReference = new AtomicReference<>(State.initial());
return new HiveSplitSource(
session,
databaseName,
tableName,
compactEffectivePredicate,
new PerBucket()
{
private final Map<Integer, AsyncQueue<InternalHiveSplit>> queues = new ConcurrentHashMap<>();
private final AtomicBoolean finished = new AtomicBoolean();
@Override
public ListenableFuture<?> offer(OptionalInt bucketNumber, InternalHiveSplit connectorSplit)
{
AsyncQueue<InternalHiveSplit> queue = queueFor(bucketNumber);
queue.offer(connectorSplit);
// Do not block "offer" when running split discovery in bucketed mode.
// A limit is enforced on estimatedSplitSizeInBytes.
return immediateFuture(null);
}
@Override
public <O> ListenableFuture<O> borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function<List<InternalHiveSplit>, BorrowResult<InternalHiveSplit, O>> function)
{
return queueFor(bucketNumber).borrowBatchAsync(maxSize, function);
}
@Override
public void finish()
{
if (finished.compareAndSet(false, true)) {
queues.values().forEach(AsyncQueue::finish);
}
}
@Override
public boolean isFinished(OptionalInt bucketNumber)
{
return queueFor(bucketNumber).isFinished();
}
public AsyncQueue<InternalHiveSplit> queueFor(OptionalInt bucketNumber)
{
checkArgument(bucketNumber.isPresent());
AtomicBoolean isNew = new AtomicBoolean();
AsyncQueue<InternalHiveSplit> queue = queues.computeIfAbsent(bucketNumber.getAsInt(), ignored -> {
isNew.set(true);
return new AsyncQueue<>(estimatedOutstandingSplitsPerBucket, executor);
});
if (isNew.get() && finished.get()) {
// Check `finished` and invoke `queue.finish` after the `queue` is added to the map.
// Otherwise, `queue.finish` may not be invoked if `finished` is set while the lambda above is being evaluated.
queue.finish();
}
return queue;
}
},
maxInitialSplits,
maxOutstandingSplitsSize,
splitLoader,
stateReference,
highMemorySplitSourceCounter);
}
/**
* The upper bound of outstanding split count.
* It might be larger than the actual number when called concurrently with other methods.
*/
@VisibleForTesting
int getBufferedInternalSplitCount()
{
return bufferedInternalSplitCount.get();
}
ListenableFuture<?> addToQueue(List<? extends InternalHiveSplit> splits)
{
ListenableFuture<?> lastResult = immediateFuture(null);
for (InternalHiveSplit split : splits) {
lastResult = addToQueue(split);
}
return lastResult;
}
ListenableFuture<?> addToQueue(InternalHiveSplit split)
{
if (stateReference.get().getKind() != INITIAL) {
return immediateFuture(null);
}
if (estimatedSplitSizeInBytes.addAndGet(split.getEstimatedSizeInBytes()) > maxOutstandingSplitsBytes) {
// TODO: investigate alternative split discovery strategies when this error is hit.
// This limit should never be hit given there is a limit of maxOutstandingSplits.
// If it's hit, it means individual splits are huge.
if (loggedHighMemoryWarning.compareAndSet(false, true)) {
highMemorySplitSourceCounter.update(1);
log.warn("Split buffering for %s.%s in query %s exceeded memory limit (%s). %s splits are buffered.",
databaseName, tableName, queryId, succinctBytes(maxOutstandingSplitsBytes), getBufferedInternalSplitCount());
}
throw new PrestoException(HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT, format(
"Split buffering for %s.%s exceeded memory limit (%s). %s splits are buffered.",
databaseName, tableName, succinctBytes(maxOutstandingSplitsBytes), getBufferedInternalSplitCount()));
}
bufferedInternalSplitCount.incrementAndGet();
OptionalInt bucketNumber = split.getBucketNumber();
return queues.offer(bucketNumber, split);
}
void noMoreSplits()
{
if (setIf(stateReference, State.noMoreSplits(), state -> state.getKind() == INITIAL)) {
// Stop the split loader before finishing the queue.
// Once the queue is finished, it will always return a completed future to avoid blocking any caller.
// This could lead to a short period of busy loop in splitLoader (although unlikely in general setup).
splitLoader.stop();
queues.finish();
}
}
void fail(Throwable e)
{
// The error must be recorded before setting the finish marker to make sure
// isFinished will observe failure instead of successful completion.
// Only record the first error message.
if (setIf(stateReference, State.failed(e), state -> state.getKind() == INITIAL)) {
// Stop the split loader before finishing the queue.
// Once the queue is finished, it will always return a completed future to avoid blocking any caller.
// This could lead to a short period of busy loop in splitLoader (although unlikely in general setup).
splitLoader.stop();
queues.finish();
}
}
@Override
public CompletableFuture<ConnectorSplitBatch> getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize)
{
boolean noMoreSplits;
State state = stateReference.get();
switch (state.getKind()) {
case INITIAL:
noMoreSplits = false;
break;
case NO_MORE_SPLITS:
noMoreSplits = true;
break;
case FAILED:
return failedFuture(state.getThrowable());
case CLOSED:
throw new IllegalStateException("HiveSplitSource is already closed");
default:
throw new UnsupportedOperationException();
}
OptionalInt bucketNumber = toBucketNumber(partitionHandle);
ListenableFuture<List<ConnectorSplit>> future = queues.borrowBatchAsync(bucketNumber, maxSize, internalSplits -> {
ImmutableList.Builder<InternalHiveSplit> splitsToInsertBuilder = ImmutableList.builder();
ImmutableList.Builder<ConnectorSplit> resultBuilder = ImmutableList.builder();
int removedEstimatedSizeInBytes = 0;
for (InternalHiveSplit internalSplit : internalSplits) {
long maxSplitBytes = maxSplitSize.toBytes();
if (remainingInitialSplits.get() > 0) {
if (remainingInitialSplits.getAndDecrement() > 0) {
maxSplitBytes = maxInitialSplitSize.toBytes();
}
}
InternalHiveBlock block = internalSplit.currentBlock();
long splitBytes;
if (internalSplit.isSplittable()) {
splitBytes = min(maxSplitBytes, block.getEnd() - internalSplit.getStart());
}
else {
splitBytes = internalSplit.getEnd() - internalSplit.getStart();
}
resultBuilder.add(new HiveSplit(
databaseName,
tableName,
internalSplit.getPartitionName(),
internalSplit.getPath(),
internalSplit.getStart(),
splitBytes,
internalSplit.getFileSize(),
internalSplit.getSchema(),
internalSplit.getPartitionKeys(),
block.getAddresses(),
internalSplit.getBucketNumber(),
internalSplit.isForceLocalScheduling(),
(TupleDomain<HiveColumnHandle>) compactEffectivePredicate,
transformValues(internalSplit.getColumnCoercions(), HiveTypeName::toHiveType),
internalSplit.getBucketConversion()));
internalSplit.increaseStart(splitBytes);
if (internalSplit.isDone()) {
removedEstimatedSizeInBytes += internalSplit.getEstimatedSizeInBytes();
}
else {
splitsToInsertBuilder.add(internalSplit);
}
}
estimatedSplitSizeInBytes.addAndGet(-removedEstimatedSizeInBytes);
List<InternalHiveSplit> splitsToInsert = splitsToInsertBuilder.build();
List<ConnectorSplit> result = resultBuilder.build();
bufferedInternalSplitCount.addAndGet(splitsToInsert.size() - result.size());
return new AsyncQueue.BorrowResult<>(splitsToInsert, result);
});
ListenableFuture<ConnectorSplitBatch> transform = Futures.transform(future, splits -> {
requireNonNull(splits, "splits is null");
if (noMoreSplits) {
// Checking splits.isEmpty() here is required for thread safety.
// Let's say there are 10 splits left, and max number of splits per batch is 5.
// The futures constructed in two getNextBatch calls could each fetch 5, resulting in zero splits left.
// After fetching the splits, both futures reach this line at the same time.
// Without the isEmpty check, both will claim they are the last.
// Side note 1: In such a case, it doesn't actually matter which one gets to claim it's the last.
// But having both claim they are the last would be a surprising behavior.
// Side note 2: One could argue that the isEmpty check is overly conservative.
// The caller of getNextBatch will likely need to make an extra invocation.
// But an extra invocation likely doesn't matter.
return new ConnectorSplitBatch(splits, splits.isEmpty() && queues.isFinished(bucketNumber));
}
else {
return new ConnectorSplitBatch(splits, false);
}
}, directExecutor());
return toCompletableFuture(transform);
}
@Override
public boolean isFinished()
{
State state = stateReference.get();
switch (state.getKind()) {
case INITIAL:
return false;
case NO_MORE_SPLITS:
return bufferedInternalSplitCount.get() == 0;
case FAILED:
throw propagatePrestoException(state.getThrowable());
case CLOSED:
throw new IllegalStateException("HiveSplitSource is already closed");
default:
throw new UnsupportedOperationException();
}
}
@Override
public void close()
{
if (setIf(stateReference, State.closed(), state -> state.getKind() == INITIAL || state.getKind() == NO_MORE_SPLITS)) {
// Stop the split loader before finishing the queue.
// Once the queue is finished, it will always return a completed future to avoid blocking any caller.
// This could lead to a short period of busy loop in splitLoader (although unlikely in general setup).
splitLoader.stop();
queues.finish();
}
}
private static OptionalInt toBucketNumber(ConnectorPartitionHandle partitionHandle)
{
if (partitionHandle == NOT_PARTITIONED) {
return OptionalInt.empty();
}
return OptionalInt.of(((HivePartitionHandle) partitionHandle).getBucket());
}
private static <T> boolean setIf(AtomicReference<T> atomicReference, T newValue, Predicate<T> predicate)
{
while (true) {
T current = atomicReference.get();
if (!predicate.test(current)) {
return false;
}
if (atomicReference.compareAndSet(current, newValue)) {
return true;
}
}
}
private static RuntimeException propagatePrestoException(Throwable throwable)
{
if (throwable instanceof PrestoException) {
throw (PrestoException) throwable;
}
if (throwable instanceof FileNotFoundException) {
throw new PrestoException(HIVE_FILE_NOT_FOUND, throwable);
}
throw new PrestoException(HIVE_UNKNOWN_ERROR, throwable);
}
interface PerBucket
{
ListenableFuture<?> offer(OptionalInt bucketNumber, InternalHiveSplit split);
<O> ListenableFuture<O> borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function<List<InternalHiveSplit>, BorrowResult<InternalHiveSplit, O>> function);
void finish();
boolean isFinished(OptionalInt bucketNumber);
}
static class State
{
private final StateKind kind;
private final Throwable throwable;
private State(StateKind kind, Throwable throwable)
{
this.kind = kind;
this.throwable = throwable;
}
public StateKind getKind()
{
return kind;
}
public Throwable getThrowable()
{
checkState(throwable != null);
return throwable;
}
public static State initial()
{
return new State(INITIAL, null);
}
public static State noMoreSplits()
{
return new State(NO_MORE_SPLITS, null);
}
public static State failed(Throwable throwable)
{
return new State(FAILED, throwable);
}
public static State closed()
{
return new State(CLOSED, null);
}
}
enum StateKind
{
INITIAL,
NO_MORE_SPLITS,
FAILED,
CLOSED,
}
}