aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
blob: b4fcedf0face2dbb46f81bc4b6983b1f9e5c9b42 (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
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
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
/*
 * 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.test;

import java.io.BufferedReader;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileReader;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

import org.apache.drill.common.exceptions.UserException;
import org.apache.drill.common.exceptions.UserRemoteException;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.exec.client.LoggingResultsListener;
import org.apache.drill.exec.client.QuerySubmitter.Format;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
import org.apache.drill.exec.proto.UserBitShared.QueryType;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.record.RecordBatchLoader;
import org.apache.drill.exec.record.VectorContainer;
import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.rpc.ConnectionThrottle;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
import org.apache.drill.exec.rpc.user.QueryDataBatch;
import org.apache.drill.exec.rpc.user.UserResultsListener;
import org.apache.drill.exec.util.VectorUtil;
import org.apache.drill.exec.vector.NullableVarCharVector;
import org.apache.drill.exec.vector.ValueVector;
import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
import org.apache.drill.test.ClientFixture.StatementParser;
import org.apache.drill.test.rowSet.DirectRowSet;
import org.apache.drill.test.rowSet.RowSet;
import org.apache.drill.test.rowSet.RowSetReader;

import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;

/**
 * Builder for a Drill query. Provides all types of query formats,
 * and a variety of ways to run the query.
 */

public class QueryBuilder {

  /**
   * Listener used to retrieve the query summary (only) asynchronously
   * using a {@link QuerySummaryFuture}.
   */

  public class SummaryOnlyQueryEventListener implements UserResultsListener {

    /**
     * The future to be notified. Created here and returned by the
     * query builder.
     */

    private final QuerySummaryFuture future;
    private QueryId queryId;
    private int recordCount;
    private int batchCount;
    private long startTime;

    public SummaryOnlyQueryEventListener(QuerySummaryFuture future) {
      this.future = future;
      startTime = System.currentTimeMillis();
    }

    @Override
    public void queryIdArrived(QueryId queryId) {
      this.queryId = queryId;
    }

    @Override
    public void submissionFailed(UserException ex) {
      future.completed(
          new QuerySummary(queryId, recordCount, batchCount,
                           System.currentTimeMillis() - startTime, ex));
    }

    @Override
    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
      batchCount++;
      recordCount += result.getHeader().getRowCount();
      result.release();
    }

    @Override
    public void queryCompleted(QueryState state) {
      future.completed(
          new QuerySummary(queryId, recordCount, batchCount,
                           System.currentTimeMillis() - startTime, state));
    }
  }

  /**
   * The future used to wait for the completion of an async query. Returns
   * just the summary of the query.
   */

  public class QuerySummaryFuture implements Future<QuerySummary> {

    /**
     * Synchronizes the listener thread and the test thread that
     * launched the query.
     */

    private CountDownLatch lock = new CountDownLatch(1);
    private QuerySummary summary;

    /**
     * Unsupported at present.
     */

    @Override
    public boolean cancel(boolean mayInterruptIfRunning) {
      throw new UnsupportedOperationException();
    }

    /**
     * Always returns false.
     */

    @Override
    public boolean isCancelled() { return false; }

    @Override
    public boolean isDone() { return summary != null; }

    @Override
    public QuerySummary get() throws InterruptedException, ExecutionException {
      lock.await();
      return summary;
    }

    /**
     * Not supported at present, just does a non-timeout get.
     */

    @Override
    public QuerySummary get(long timeout, TimeUnit unit)
        throws InterruptedException, ExecutionException, TimeoutException {
      return get();
    }

    protected void completed(QuerySummary querySummary) {
      summary = querySummary;
      lock.countDown();
    }
  }

  /**
   * Summary results of a query: records, batches, run time.
   */

  public static class QuerySummary {
    private final QueryId queryId;
    private final int records;
    private final int batches;
    private final long ms;
    private final QueryState finalState;
    private final Exception error;

    public QuerySummary(QueryId queryId, int recordCount, int batchCount, long elapsed, QueryState state) {
      this.queryId = queryId;
      records = recordCount;
      batches = batchCount;
      ms = elapsed;
      finalState = state;
      error = null;
    }

    public QuerySummary(QueryId queryId, int recordCount, int batchCount, long elapsed, Exception ex) {
      this.queryId = queryId;
      records = recordCount;
      batches = batchCount;
      ms = elapsed;
      finalState = null;
      error = ex;
    }

    public boolean failed() { return error != null; }
    public boolean succeeded() { return error == null; }
    public long recordCount() { return records; }
    public int batchCount() { return batches; }
    public long runTimeMs() { return ms; }
    public QueryId queryId() { return queryId; }
    public String queryIdString() { return QueryIdHelper.getQueryId(queryId); }
    public Exception error() { return error; }
    public QueryState finalState() { return finalState; }
  }

  private final ClientFixture client;
  private QueryType queryType;
  private String queryText;
  private List<PlanFragment> planFragments;

  QueryBuilder(ClientFixture client) {
    this.client = client;
  }

  public QueryBuilder query(QueryType type, String text) {
    queryType = type;
    queryText = text;
    return this;
  }

  public QueryBuilder sql(String sql) {
    return query(QueryType.SQL, sql);
  }

  public QueryBuilder sql(String query, Object... args) {
    return sql(String.format(query, args));
  }

  /**
   * Run a physical plan presented as a list of fragments.
   *
   * @param planFragments fragments that make up the plan
   * @return this builder
   */

  public QueryBuilder plan(List<PlanFragment> planFragments) {
    queryType = QueryType.EXECUTION;
    this.planFragments = planFragments;
    return this;
  }

  /**
   * Parse a single SQL statement (with optional ending semi-colon) from
   * the file provided.
   * @param file the file containing exactly one SQL statement, with
   * optional ending semi-colon
   * @return this builder
   */

  public QueryBuilder sql(File file) throws FileNotFoundException, IOException {
    try (BufferedReader in = new BufferedReader(new FileReader(file))) {
      StatementParser parser = new StatementParser(in);
      String sql = parser.parseNext();
      if (sql == null) {
        throw new IllegalArgumentException("No query found");
      }
      return sql(sql);
    }
  }

  public QueryBuilder physical(String plan) {
    return query(QueryType.PHYSICAL, plan);
  }

  /**
   * Run a query contained in a resource file.
   *
   * @param resource Name of the resource
   * @return this builder
   */

  public QueryBuilder sqlResource(String resource) {
    sql(ClusterFixture.loadResource(resource));
    return this;
  }

  public QueryBuilder sqlResource(String resource, Object... args) {
    sql(ClusterFixture.loadResource(resource), args);
    return this;
  }

  public QueryBuilder physicalResource(String resource) {
    physical(ClusterFixture.loadResource(resource));
    return this;
  }

  /**
   * Run the query returning just a summary of the results: record count,
   * batch count and run time. Handy when doing performance tests when the
   * validity of the results is verified in some other test.
   *
   * @return the query summary
   * @throws Exception if anything goes wrong anywhere in the execution
   */

  public QuerySummary run() throws Exception {
    return produceSummary(withEventListener());
  }

  /**
   * Run the query and return a list of the result batches. Use
   * if the batch count is small and you want to work with them.
   * @return a list of batches resulting from the query
   * @throws RpcException
   */

  public List<QueryDataBatch> results() throws RpcException {
    Preconditions.checkNotNull(queryType, "Query not provided.");
    Preconditions.checkNotNull(queryText, "Query not provided.");
    return client.client().runQuery(queryType, queryText);
  }

  /**
   * Run the query and return the first non-empty batch as a
   * {@link DirectRowSet} object that can be inspected directly
   * by the code using a {@link RowSetReader}.
   * <p>
   *
   * @see {@link #rowSetIterator()} for a version that reads a series of
   * batches as row sets.
   * @return a row set that represents the first non-empty batch returned from
   * the query
   * @throws RpcException if anything goes wrong
   */

  public DirectRowSet rowSet() throws RpcException {

    // Ignore all but the first non-empty batch.

    QueryDataBatch dataBatch = null;
    for (QueryDataBatch batch : results()) {
      if (dataBatch == null  &&  batch.getHeader().getRowCount() != 0) {
        dataBatch = batch;
      } else {
        batch.release();
      }
    }

    // No results?

    if (dataBatch == null) {
      return null;
    }

    // Unload the batch and convert to a row set.

    final RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
    try {
      loader.load(dataBatch.getHeader().getDef(), dataBatch.getData());
      dataBatch.release();
      VectorContainer container = loader.getContainer();
      container.setRecordCount(loader.getRecordCount());
      return DirectRowSet.fromContainer(container);
    } catch (SchemaChangeException e) {
      throw new IllegalStateException(e);
    }
  }

  public QueryRowSetIterator rowSetIterator( ) {
    return new QueryRowSetIterator(client.allocator(), withEventListener());
  }

  /**
   * Run the query that is expected to return (at least) one row
   * with the only (or first) column returning a long value.
   * The long value cannot be null.
   *
   * @return the value of the first column of the first row
   * @throws RpcException if anything goes wrong
   */

  public long singletonLong() throws RpcException {
    RowSet rowSet = rowSet();
    if (rowSet == null) {
      throw new IllegalStateException("No rows returned");
    }
    RowSetReader reader = rowSet.reader();
    reader.next();
    long value = reader.scalar(0).getLong();
    rowSet.clear();
    return value;
  }

  /**
   * Run the query that is expected to return (at least) one row
   * with the only (or first) column returning a double value.
   * The double value cannot be null.
   *
   * @return the value of the first column of the first row
   * @throws RpcException if anything goes wrong
   */

  public double singletonDouble() throws RpcException {
    RowSet rowSet = rowSet();
    if (rowSet == null) {
      throw new IllegalStateException("No rows returned");
    }
    RowSetReader reader = rowSet.reader();
    reader.next();
    double value = reader.scalar(0).getDouble();
    rowSet.clear();
    return value;
  }

  /**
   * Run the query that is expected to return (at least) one row
   * with the only (or first) column returning a int value.
   * The int value cannot be null.
   *
   * @return the value of the first column of the first row
   * @throws RpcException if anything goes wrong
   */

  public int singletonInt() throws RpcException {
    RowSet rowSet = rowSet();
    if (rowSet == null) {
      throw new IllegalStateException("No rows returned");
    }
    RowSetReader reader = rowSet.reader();
    reader.next();
    int value = reader.scalar(0).getInt();
    rowSet.clear();
    return value;
  }

  /**
   * Run the query that is expected to return (at least) one row
   * with the only (or first) column returning a string value.
   * The value may be null, in which case a null string is returned.
   *
   * @return the value of the first column of the first row
   * @throws RpcException if anything goes wrong
   */

  public String singletonString() throws RpcException {
    RowSet rowSet = rowSet();
    if (rowSet == null) {
      throw new IllegalStateException("No rows returned");
    }
    RowSetReader reader = rowSet.reader();
    reader.next();
    String value;
    if (reader.scalar(0).isNull()) {
      value = null;
    } else {
      value = reader.scalar(0).getString();
    }
    rowSet.clear();
    return value;
  }

  /**
   * Run the query with the listener provided. Use when the result
   * count will be large, or you don't need the results.
   *
   * @param listener the Drill listener
   */

  public void withListener(UserResultsListener listener) {
    Preconditions.checkNotNull(queryType, "Query not provided.");
    if (planFragments != null) {
      try {
        client.client().runQuery(QueryType.EXECUTION, planFragments, listener);
      } catch(RpcException e) {
        throw new IllegalStateException(e);
      }
    } else {
      Preconditions.checkNotNull(queryText, "Query not provided.");
      client.client().runQuery(queryType, queryText, listener);
    }
  }

  /**
   * Run the query, return an easy-to-use event listener to process
   * the query results. Use when the result set is large. The listener
   * allows the caller to iterate over results in the test thread.
   * (The listener implements a producer-consumer model to hide the
   * details of Drill listeners.)
   *
   * @return the query event listener
   */

  public BufferingQueryEventListener withEventListener() {
    BufferingQueryEventListener listener = new BufferingQueryEventListener();
    withListener(listener);
    return listener;
  }

  public long logCsv() {
    return log(Format.CSV);
  }

  public long log(Format format) {
    return log(format,20);
  }

  public long log(Format format, int colWidth) {
    return runAndWait(new LoggingResultsListener(client.cluster().config(), format, colWidth));
  }

  /**
   * <p>
   *   Run a query and logs the output in TSV format.
   *   Similar to {@link QueryTestUtil#testRunAndLog} with one query.
   * </p>
   *
   * @return The number of rows returned.
   * @throws Exception If anything goes wrong with query execution.
   */
  public long log() throws Exception {
    return log(Format.TSV, VectorUtil.DEFAULT_COLUMN_WIDTH);
  }

  public long printCsv() {
    return print(Format.CSV);
  }

  public long print(Format format) {
    return print(format,20);
  }

  public long print(Format format, int colWidth) {
    return runAndWait(new PrintingResultsListener(client.cluster().config(), format, colWidth));
  }

  /**
   * <p>
   *   Runs a query and prints the output to stdout in TSV format.
   *   Similar to {@link QueryTestUtil#testRunAndLog} with one query.
   * </p>
   *
   * @return The number of rows returned.
   * @throws Exception If anything goes wrong with query execution.
   */
  public long print() throws Exception {
    return print(Format.TSV, VectorUtil.DEFAULT_COLUMN_WIDTH);
  }

  /**
   * Run the query asynchronously, returning a future to be used
   * to check for query completion, wait for completion, and obtain
   * the result summary.
   */

  public QuerySummaryFuture futureSummary() {
    QuerySummaryFuture future = new QuerySummaryFuture();
    withListener(new SummaryOnlyQueryEventListener(future));
    return future;
  }

  public long runAndWait(UserResultsListener listener) {
    AwaitableUserResultsListener resultListener =
        new AwaitableUserResultsListener(listener);
    withListener(resultListener);
    try {
      return resultListener.await();
    } catch (UserRemoteException e) {
      throw e;
    } catch (Exception e) {
      throw new IllegalStateException(e);
    }
  }

  /**
   * Submit an "EXPLAIN" statement, and return text form of the
   * plan.
   * @throws Exception if the query fails
   */

  public String explainText() throws Exception {
    return explain(ClusterFixture.EXPLAIN_PLAN_TEXT);
  }

  /**
   * Submit an "EXPLAIN" statement, and return the JSON form of the
   * plan.
   * @throws Exception if the query fails
   */

  public String explainJson() throws Exception {
    return explain(ClusterFixture.EXPLAIN_PLAN_JSON);
  }

  public String explain(String format) throws Exception {
    queryText = "EXPLAIN PLAN FOR " + queryText;
    return queryPlan(format);
  }

  private QuerySummary produceSummary(BufferingQueryEventListener listener) throws Exception {
    long start = System.currentTimeMillis();
    int recordCount = 0;
    int batchCount = 0;
    QueryId queryId = null;
    QueryState state = null;
    loop:
    for (;;) {
      QueryEvent event = listener.get();
      switch (event.type)
      {
      case BATCH:
        batchCount++;
        recordCount += event.batch.getHeader().getRowCount();
        event.batch.release();
        break;
      case EOF:
        state = event.state;
        break loop;
      case ERROR:
        throw event.error;
      case QUERY_ID:
        queryId = event.queryId;
        break;
      default:
        throw new IllegalStateException("Unexpected event: " + event.type);
      }
    }
    long end = System.currentTimeMillis();
    long elapsed = end - start;
    return new QuerySummary(queryId, recordCount, batchCount, elapsed, state);
  }

  public QueryResultSet resultSet() {
    BufferingQueryEventListener listener = withEventListener();
    return new QueryResultSet(listener, client.allocator());
  }

  /**
   * Submit an "EXPLAIN" statement, and return the column value which
   * contains the plan's string.
   * <p>
   * Cribbed from {@link PlanTestBase#getPlanInString(String, String)}
   * @throws Exception if anything goes wrogn in the query
   */

  protected String queryPlan(String columnName) throws Exception {
    Preconditions.checkArgument(queryType == QueryType.SQL, "Can only explan an SQL query.");
    final List<QueryDataBatch> results = results();
    final RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
    final StringBuilder builder = new StringBuilder();

    for (final QueryDataBatch b : results) {
      if (!b.hasData()) {
        continue;
      }

      loader.load(b.getHeader().getDef(), b.getData());

      final VectorWrapper<?> vw;
      try {
          vw = loader.getValueAccessorById(
              NullableVarCharVector.class,
              loader.getValueVectorId(SchemaPath.getSimplePath(columnName)).getFieldIds());
      } catch (Throwable t) {
        throw new IllegalStateException("Looks like you did not provide an explain plan query, please add EXPLAIN PLAN FOR to the beginning of your query.");
      }

      @SuppressWarnings("resource")
      final ValueVector vv = vw.getValueVector();
      for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
        final Object o = vv.getAccessor().getObject(i);
        builder.append(o);
      }
      loader.clear();
      b.release();
    }

    return builder.toString();
  }
}