aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
blob: 50f582dfa38e4121df543d98e2d9b14188545970 (plain)
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
/*
 * 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.drill.exec.work.batch;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.DrillBuf;

import java.io.EOFException;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;

import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.proto.BitData;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.record.RawFragmentBatch;
import org.apache.drill.exec.store.LocalSyncableFileSystem;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
import org.apache.drill.shaded.guava.com.google.common.collect.Queues;

/**
 * This implementation of RawBatchBuffer starts writing incoming batches to disk once the buffer size reaches a threshold.
 * The order of the incoming buffers is maintained.
 */
public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchBuffer.RawFragmentBatchWrapper> {
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SpoolingRawBatchBuffer.class);

  private static String DRILL_LOCAL_IMPL_STRING = "fs.drill-local.impl";
  private static final float STOP_SPOOLING_FRACTION = (float) 0.5;
  public static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024;
  public static final long ALLOCATOR_MAX_RESERVATION = 20L*1000*1000*1000;

  private enum SpoolingState {
    NOT_SPOOLING,
    SPOOLING,
    PAUSE_SPOOLING,
    STOP_SPOOLING
  }

  private final BufferAllocator allocator;
  private final long threshold;
  private final int oppositeId;
  private final int bufferIndex;

  private volatile SpoolingState spoolingState;
  private volatile long currentSizeInMemory = 0;
  private volatile Spooler spooler;

  private FileSystem fs;
  private Path path;
  private FSDataOutputStream outputStream;

  public SpoolingRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId, int bufferIndex) {
    super(context, fragmentCount);
    this.allocator = context.getNewChildAllocator(
        "SpoolingRawBatchBufer", 100, ALLOCATOR_INITIAL_RESERVATION, ALLOCATOR_MAX_RESERVATION);
    this.threshold = context.getConfig().getLong(ExecConstants.SPOOLING_BUFFER_MEMORY);
    this.oppositeId = oppositeId;
    this.bufferIndex = bufferIndex;
    this.bufferQueue = new SpoolingBufferQueue();
  }

  private class SpoolingBufferQueue implements BufferQueue<RawFragmentBatchWrapper> {

    private final LinkedBlockingDeque<RawFragmentBatchWrapper> buffer = Queues.newLinkedBlockingDeque();

    @Override
    public void addOomBatch(RawFragmentBatch batch) {
      RawFragmentBatchWrapper batchWrapper = new RawFragmentBatchWrapper(batch, true);
      batchWrapper.setOutOfMemory(true);
      buffer.addFirst(batchWrapper);
    }

    @Override
    public RawFragmentBatch poll() throws IOException, InterruptedException {
      RawFragmentBatchWrapper batchWrapper = buffer.poll();
      if (batchWrapper != null) {
        return batchWrapper.get();
      }
      return null;
    }

    @Override
    public RawFragmentBatch take() throws IOException, InterruptedException {
      return buffer.take().get();
    }

    @Override
    public RawFragmentBatch poll(long timeout, TimeUnit timeUnit) throws InterruptedException, IOException {
      RawFragmentBatchWrapper batchWrapper = buffer.poll(timeout, timeUnit);
      if (batchWrapper != null) {
        return batchWrapper.get();
      }
      return null;
    }

    @Override
    public boolean checkForOutOfMemory() {
      return buffer.peek().isOutOfMemory();
    }

    @Override
    public int size() {
      return buffer.size();
    }

    @Override
    public boolean isEmpty() {
      return buffer.size() == 0;
    }

    public void add(RawFragmentBatchWrapper batchWrapper) {
      buffer.add(batchWrapper);
    }
  }

  private synchronized void setSpoolingState(SpoolingState newState) {
    SpoolingState currentState = spoolingState;
    if (newState == SpoolingState.NOT_SPOOLING ||
        currentState == SpoolingState.STOP_SPOOLING) {
      return;
    }
    spoolingState = newState;
  }

  private boolean isCurrentlySpooling() {
    return spoolingState == SpoolingState.SPOOLING;
  }

  private void startSpooling() {
    setSpoolingState(SpoolingState.SPOOLING);
  }

  private void pauseSpooling() {
    setSpoolingState(SpoolingState.PAUSE_SPOOLING);
  }

  private boolean isSpoolingStopped() {
    return spoolingState == SpoolingState.STOP_SPOOLING;
  }

  private void stopSpooling() {
    setSpoolingState(SpoolingState.STOP_SPOOLING);
  }

  public String getDir() {
    List<String> dirs = context.getConfig().getStringList(ExecConstants.TEMP_DIRECTORIES);
    return dirs.get(ThreadLocalRandom.current().nextInt(dirs.size()));
  }

  private synchronized void initSpooler() throws IOException {
    if (spooler != null) {
      return;
    }

    Configuration conf = new Configuration();
    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, context.getConfig().getString(ExecConstants.TEMP_FILESYSTEM));
    conf.set(DRILL_LOCAL_IMPL_STRING, LocalSyncableFileSystem.class.getName());
    fs = FileSystem.get(conf);
    path = getPath();
    outputStream = fs.create(path);
    final String spoolingThreadName = QueryIdHelper.getExecutorThreadName(context.getHandle()).concat(
        ":Spooler-" + oppositeId + "-" + bufferIndex);
    spooler = new Spooler(spoolingThreadName);
    spooler.start();
  }

  @Override
  protected void enqueueInner(RawFragmentBatch batch) throws IOException {
    assert batch.getHeader().getSendingMajorFragmentId() == oppositeId;

    logger.debug("Enqueue batch. Current buffer size: {}. Last batch: {}. Sending fragment: {}", bufferQueue.size(), batch.getHeader().getIsLastBatch(), batch.getHeader().getSendingMajorFragmentId());
    RawFragmentBatchWrapper wrapper;

    boolean spoolCurrentBatch = isCurrentlySpooling();
    wrapper = new RawFragmentBatchWrapper(batch, !spoolCurrentBatch);
    currentSizeInMemory += wrapper.getBodySize();
    if (spoolCurrentBatch) {
      if (spooler == null) {
        initSpooler();
      }
      spooler.addBatchForSpooling(wrapper);
    }
    bufferQueue.add(wrapper);
    if (!spoolCurrentBatch && currentSizeInMemory > threshold) {
      logger.debug("Buffer size {} greater than threshold {}. Start spooling to disk", currentSizeInMemory, threshold);
      startSpooling();
    }
  }

  @Override
  public void kill(FragmentContext context) {
    allocator.close();
    if (spooler != null) {
      spooler.terminate();
    }
  }

  @Override
  protected void upkeep(RawFragmentBatch batch) {
    if (context.getAllocator().isOverLimit()) {
      outOfMemory.set(true);
    }

    DrillBuf body = batch.getBody();
    if (body != null) {
      currentSizeInMemory -= body.capacity();
    }
    if (isCurrentlySpooling() && currentSizeInMemory < threshold * STOP_SPOOLING_FRACTION) {
      logger.debug("buffer size {} less than {}x threshold. Stop spooling.", currentSizeInMemory, STOP_SPOOLING_FRACTION);
      pauseSpooling();
    }
    logger.debug("Got batch. Current buffer size: {}", bufferQueue.size());
  }

  @Override
  public void close() {
    if (spooler != null) {
      spooler.terminate();
      while (spooler.isAlive()) {
        try {
          spooler.join();
        } catch (InterruptedException e) {
          logger.warn("Interrupted while waiting for spooling thread to exit");
          continue;
        }
      }
    }
    allocator.close();
    try {
      if (outputStream != null) {
        outputStream.close();
      }
    } catch (IOException e) {
      logger.warn("Failed to cleanup I/O streams", e);
    }
    if (context.getConfig().getBoolean(ExecConstants.SPOOLING_BUFFER_DELETE)) {
      try {
        if (fs != null) {
          fs.delete(path, false);
          logger.debug("Deleted file {}", path.toString());
        }
      } catch (IOException e) {
        logger.warn("Failed to delete temporary files", e);
      }
    }
    super.close();
  }

  private class Spooler extends Thread {

    private final LinkedBlockingDeque<RawFragmentBatchWrapper> spoolingQueue;
    private volatile boolean shouldContinue = true;
    private Thread spoolingThread;

    public Spooler(String name) {
      setDaemon(true);
      setName(name);
      spoolingQueue = Queues.newLinkedBlockingDeque();
    }

    public void run() {
      try {
        while (shouldContinue) {
          RawFragmentBatchWrapper batch;
          try {
            batch = spoolingQueue.take();
          } catch (InterruptedException e) {
            if (shouldContinue) {
              continue;
            } else {
              break;
            }
          }
          try {
            batch.writeToStream(outputStream);
          } catch (IOException e) {
            context.getExecutorState().fail(e);
          }
        }
      } catch (Throwable e) {
        context.getExecutorState().fail(e);
      } finally {
        logger.info("Spooler thread exiting");
      }
    }

    public void addBatchForSpooling(RawFragmentBatchWrapper batchWrapper) {
      if (isSpoolingStopped()) {
        spoolingQueue.add(batchWrapper);
      } else {
        // will not spill this batch
        batchWrapper.available = true;
        batchWrapper.batch.sendOk();
        batchWrapper.latch.countDown();
      }
    }

    public void terminate() {
      stopSpooling();
      shouldContinue = false;
      if (spoolingThread.isAlive()) {
        spoolingThread.interrupt();
      }
    }
  }

  class RawFragmentBatchWrapper {
    private RawFragmentBatch batch;
    private volatile boolean available;
    private CountDownLatch latch;
    private volatile int bodyLength;
    private volatile boolean outOfMemory = false;
    private long start = -1;
    private long check;

    public RawFragmentBatchWrapper(RawFragmentBatch batch, boolean available) {
      Preconditions.checkNotNull(batch);
      this.batch = batch;
      this.available = available;
      this.latch = new CountDownLatch(available ? 0 : 1);
      if (available) {
        batch.sendOk();
      }
    }

    public boolean isNull() {
      return batch == null;
    }

    public RawFragmentBatch get() throws InterruptedException, IOException {
      if (available) {
        assert batch.getHeader() != null : "batch header null";
        return batch;
      } else {
        latch.await();
        readFromStream();
        available = true;
        return batch;
      }
    }

    public long getBodySize() {
      if (batch.getBody() == null) {
        return 0;
      }
      assert batch.getBody().readableBytes() >= 0;
      return batch.getBody().readableBytes();
    }

    public void writeToStream(FSDataOutputStream stream) throws IOException {
      Stopwatch watch = Stopwatch.createStarted();
      available = false;
      check = ThreadLocalRandom.current().nextLong();
      start = stream.getPos();
      logger.debug("Writing check value {} at position {}", check, start);
      stream.writeLong(check);
      batch.getHeader().writeDelimitedTo(stream);
      ByteBuf buf = batch.getBody();
      if (buf != null) {
        bodyLength = buf.capacity();
      } else {
        bodyLength = 0;
      }
      if (bodyLength > 0) {
        buf.getBytes(0, stream, bodyLength);
      }
      stream.hsync();
      FileStatus status = fs.getFileStatus(path);
      long len = status.getLen();
      logger.debug("After spooling batch, stream at position {}. File length {}", stream.getPos(), len);
      batch.sendOk();
      latch.countDown();
      long t = watch.elapsed(TimeUnit.MICROSECONDS);
      logger.debug("Took {} us to spool {} to disk. Rate {} mb/s", t, bodyLength, bodyLength / t);
      if (buf != null) {
        buf.release();
      }
    }

    public void readFromStream() throws IOException, InterruptedException {
      long pos = start;
      boolean tryAgain = true;
      int duration = 0;

      while (tryAgain) {

        // Sometimes, the file isn't quite done writing when we attempt to read it. As such, we need to wait and retry.
        Thread.sleep(duration);

        try(final FSDataInputStream stream = fs.open(path);
            final DrillBuf buf = allocator.buffer(bodyLength)) {
          stream.seek(start);
          final long currentPos = stream.getPos();
          final long check = stream.readLong();
          pos = stream.getPos();
          assert check == this.check : String.format("Check values don't match: %d %d, Position %d", this.check, check, currentPos);
          Stopwatch watch = Stopwatch.createStarted();
          BitData.FragmentRecordBatch header = BitData.FragmentRecordBatch.parseDelimitedFrom(stream);
          pos = stream.getPos();
          assert header != null : "header null after parsing from stream";
          buf.writeBytes(stream, bodyLength);
          pos = stream.getPos();
          batch = new RawFragmentBatch(header, buf, null);
          available = true;
          latch.countDown();
          long t = watch.elapsed(TimeUnit.MICROSECONDS);
          logger.debug("Took {} us to read {} from disk. Rate {} mb/s", t, bodyLength, bodyLength / t);
          tryAgain = false;
        } catch (EOFException e) {
          FileStatus status = fs.getFileStatus(path);
          logger.warn("EOF reading from file {} at pos {}. Current file size: {}", path, pos, status.getLen());
          duration = Math.max(1, duration * 2);
          if (duration < 60000) {
            continue;
          } else {
            throw e;
          }
        } finally {
          if (tryAgain) {
            // we had a premature exit, release batch memory so we don't leak it.
            if (batch != null) {
              batch.getBody().release();
            }
          }
        }
      }
    }

    private boolean isOutOfMemory() {
      return outOfMemory;
    }

    private void setOutOfMemory(boolean outOfMemory) {
      this.outOfMemory = outOfMemory;
    }
  }

  private Path getPath() {
    ExecProtos.FragmentHandle handle = context.getHandle();

    String qid = QueryIdHelper.getQueryId(handle.getQueryId());

    int majorFragmentId = handle.getMajorFragmentId();
    int minorFragmentId = handle.getMinorFragmentId();

    String fileName = Joiner.on(Path.SEPARATOR).join(getDir(), qid, majorFragmentId, minorFragmentId, oppositeId, bufferIndex);

    return new Path(fileName);
  }
}