aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
blob: c185ac7ac73fbfa43536b4cf06339ca59da8b4d3 (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
/*
 * 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.physical.impl.partitionsender;

import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerArray;

import org.apache.drill.common.expression.ErrorCollector;
import org.apache.drill.common.expression.ErrorCollectorImpl;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.exception.ClassTransformationException;
import org.apache.drill.exec.exception.OutOfMemoryException;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.ClassGenerator;
import org.apache.drill.exec.expr.CodeGenerator;
import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
import org.apache.drill.exec.ops.AccountingDataTunnel;
import org.apache.drill.exec.ops.ExchangeFragmentContext;
import org.apache.drill.exec.ops.MetricDef;
import org.apache.drill.exec.ops.OperatorStats;
import org.apache.drill.exec.ops.RootFragmentContext;
import org.apache.drill.exec.physical.MinorFragmentEndpoint;
import org.apache.drill.exec.physical.config.HashPartitionSender;
import org.apache.drill.exec.physical.impl.BaseRootExec;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.record.BatchSchema;
import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
import org.apache.drill.exec.record.CloseableRecordBatch;
import org.apache.drill.exec.record.FragmentWritableBatch;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.RecordBatch.IterOutcome;
import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.vector.CopyUtil;

import com.carrotsearch.hppc.IntArrayList;
import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
import com.sun.codemodel.JExpr;
import com.sun.codemodel.JExpression;
import com.sun.codemodel.JType;

public class PartitionSenderRootExec extends BaseRootExec {
  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionSenderRootExec.class);
  private RecordBatch incoming;
  private HashPartitionSender operator;
  private PartitionerDecorator partitioner;

  private ExchangeFragmentContext context;
  private final int outGoingBatchCount;
  private final HashPartitionSender popConfig;
  private final double cost;

  private final AtomicIntegerArray remainingReceivers;
  private final AtomicInteger remaingReceiverCount;
  private boolean done = false;
  private boolean first = true;
  private boolean closeIncoming;

  long minReceiverRecordCount = Long.MAX_VALUE;
  long maxReceiverRecordCount = Long.MIN_VALUE;
  protected final int numberPartitions;
  protected final int actualPartitions;

  private IntArrayList terminations = new IntArrayList();

  public enum Metric implements MetricDef {
    BATCHES_SENT,
    RECORDS_SENT,
    MIN_RECORDS,
    MAX_RECORDS,
    N_RECEIVERS,
    BYTES_SENT,
    SENDING_THREADS_COUNT,
    COST;

    @Override
    public int metricId() {
      return ordinal();
    }
  }

  public PartitionSenderRootExec(RootFragmentContext context,
                                 RecordBatch incoming,
                                 HashPartitionSender operator) throws OutOfMemoryException {
    this(context, incoming, operator, false);
  }

  public PartitionSenderRootExec(RootFragmentContext context,
                                 RecordBatch incoming,
                                 HashPartitionSender operator,
                                 boolean closeIncoming) throws OutOfMemoryException {
    super(context, context.newOperatorContext(operator, null), operator);
    this.incoming = incoming;
    this.operator = operator;
    this.closeIncoming = closeIncoming;
    this.context = context;
    outGoingBatchCount = operator.getDestinations().size();
    popConfig = operator;
    remainingReceivers = new AtomicIntegerArray(outGoingBatchCount);
    remaingReceiverCount = new AtomicInteger(outGoingBatchCount);
    stats.setLongStat(Metric.N_RECEIVERS, outGoingBatchCount);
    // Algorithm to figure out number of threads to parallelize output
    // numberOfRows/sliceTarget/numReceivers/threadfactor
    this.cost = operator.getChild().getCost();
    final OptionManager optMgr = context.getOptions();
    long sliceTarget = optMgr.getOption(ExecConstants.SLICE_TARGET).num_val;
    int threadFactor = optMgr.getOption(PlannerSettings.PARTITION_SENDER_THREADS_FACTOR.getOptionName()).num_val.intValue();
    int tmpParts = 1;
    if ( sliceTarget != 0 && outGoingBatchCount != 0 ) {
      tmpParts = (int) Math.round((((cost / (sliceTarget*1.0)) / (outGoingBatchCount*1.0)) / (threadFactor*1.0)));
      if ( tmpParts < 1) {
        tmpParts = 1;
      }
    }
    final int imposedThreads = optMgr.getOption(PlannerSettings.PARTITION_SENDER_SET_THREADS.getOptionName()).num_val.intValue();
    if (imposedThreads > 0 ) {
      this.numberPartitions = imposedThreads;
    } else {
      this.numberPartitions = Math.min(tmpParts, optMgr.getOption(PlannerSettings.PARTITION_SENDER_MAX_THREADS.getOptionName()).num_val.intValue());
    }
    logger.info("Preliminary number of sending threads is: " + numberPartitions);
    this.actualPartitions = outGoingBatchCount > numberPartitions ? numberPartitions : outGoingBatchCount;
    this.stats.setLongStat(Metric.SENDING_THREADS_COUNT, actualPartitions);
    this.stats.setDoubleStat(Metric.COST, this.cost);
  }

  @Override
  public boolean innerNext() {
    IterOutcome out;

    if (!done) {
      out = next(incoming);
    } else {
      incoming.kill(true);
      out = IterOutcome.NONE;
    }

    logger.debug("Partitioner.next(): got next record batch with status {}", out);
    if (first && out == IterOutcome.OK) {
      out = IterOutcome.OK_NEW_SCHEMA;
    }
    switch(out){
      case NONE:
        try {
          // send any pending batches
          if(partitioner != null) {
            partitioner.flushOutgoingBatches(true, false);
          } else {
            sendEmptyBatch(true);
          }
        } catch (ExecutionException e) {
          incoming.kill(false);
          logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
          context.getExecutorState().fail(e.getCause());
        }
        return false;

      case OUT_OF_MEMORY:
        throw new OutOfMemoryException();

      case STOP:
        if (partitioner != null) {
          partitioner.clear();
        }
        return false;

      case OK_NEW_SCHEMA:
        try {
          // send all existing batches
          if (partitioner != null) {
            partitioner.flushOutgoingBatches(false, true);
            partitioner.clear();
          }
          createPartitioner();

          if (first) {
            // Send an empty batch for fast schema
            first = false;
            sendEmptyBatch(false);
          }
        } catch (ExecutionException e) {
          incoming.kill(false);
          logger.error("Error while flushing outgoing batches", e);
          context.getExecutorState().fail(e.getCause());
          return false;
        } catch (SchemaChangeException e) {
          incoming.kill(false);
          logger.error("Error while setting up partitioner", e);
          context.getExecutorState().fail(e);
          return false;
        }
      case OK:
        try {
          partitioner.partitionBatch(incoming);
        } catch (ExecutionException e) {
          context.getExecutorState().fail(e.getCause());
          incoming.kill(false);
          return false;
        }
        for (VectorWrapper<?> v : incoming) {
          v.clear();
        }
        return true;
      case NOT_YET:
      default:
        throw new IllegalStateException();
    }
  }

  @VisibleForTesting
  protected void createPartitioner() throws SchemaChangeException {
    createClassInstances(actualPartitions);
  }

  private List<Partitioner> createClassInstances(int actualPartitions) throws SchemaChangeException {
    // set up partitioning function
    final LogicalExpression expr = operator.getExpr();
    final ErrorCollector collector = new ErrorCollectorImpl();
    final ClassGenerator<Partitioner> cg;

    cg = CodeGenerator.getRoot(Partitioner.TEMPLATE_DEFINITION, context.getOptions());
    cg.getCodeGenerator().plainJavaCapable(true);
    // Uncomment out this line to debug the generated code.
    // cg.getCodeGenerator().saveCodeForDebugging(true);
    ClassGenerator<Partitioner> cgInner = cg.getInnerGenerator("OutgoingRecordBatch");

    final LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
    if (collector.hasErrors()) {
      throw new SchemaChangeException(String.format(
          "Failure while trying to materialize incoming schema.  Errors:\n %s.",
          collector.toErrorString()));
    }

    // generate code to copy from an incoming value vector to the destination partition's outgoing value vector
    JExpression bucket = JExpr.direct("bucket");

    // generate evaluate expression to determine the hash
    ClassGenerator.HoldingContainer exprHolder = cg.addExpr(materializedExpr);
    cg.getEvalBlock().decl(JType.parse(cg.getModel(), "int"), "bucket", exprHolder.getValue().mod(JExpr.lit(outGoingBatchCount)));
    cg.getEvalBlock()._return(cg.getModel().ref(Math.class).staticInvoke("abs").arg(bucket));

    CopyUtil.generateCopies(cgInner, incoming, incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE);

    try {
      // compile and setup generated code
      List<Partitioner> subPartitioners = context.getImplementationClass(cg, actualPartitions);

      final int divisor = Math.max(1, outGoingBatchCount/actualPartitions);
      final int longTail = outGoingBatchCount % actualPartitions;
      int startIndex = 0;
      int endIndex = 0;

      boolean success = false;
      try {
        for (int i = 0; i < actualPartitions; i++) {
          startIndex = endIndex;
          endIndex = (i < actualPartitions - 1) ? startIndex + divisor : outGoingBatchCount;
          if (i < longTail) {
            endIndex++;
          }
          final OperatorStats partitionStats = new OperatorStats(stats, true);
          subPartitioners.get(i).setup(context, incoming, popConfig, partitionStats, oContext,
            cgInner, startIndex, endIndex);
        }

        partitioner = new PartitionerDecorator(subPartitioners, stats, context);
        for (int index = 0; index < terminations.size(); index++) {
          partitioner.getOutgoingBatches(terminations.buffer[index]).terminate();
        }
        terminations.clear();

        success = true;
      } finally {
        if (!success) {
          for (Partitioner p : subPartitioners) {
            p.clear();
          }
        }
      }
      return subPartitioners;

    } catch (ClassTransformationException | IOException e) {
      throw new SchemaChangeException("Failure while attempting to load generated class", e);
    }
  }

  /**
   * Find min and max record count seen across the outgoing batches and put them in stats.
   */
  private void updateAggregateStats() {
    for (Partitioner part : partitioner.getPartitioners() ) {
      for (PartitionOutgoingBatch o : part.getOutgoingBatches()) {
        long totalRecords = o.getTotalRecords();
        minReceiverRecordCount = Math.min(minReceiverRecordCount, totalRecords);
        maxReceiverRecordCount = Math.max(maxReceiverRecordCount, totalRecords);
      }
    }
    stats.setLongStat(Metric.MIN_RECORDS, minReceiverRecordCount);
    stats.setLongStat(Metric.MAX_RECORDS, maxReceiverRecordCount);
  }

  @Override
  public void receivingFragmentFinished(FragmentHandle handle) {
    final int id = handle.getMinorFragmentId();
    if (remainingReceivers.compareAndSet(id, 0, 1)) {
      if (partitioner == null) {
        terminations.add(id);
      } else {
        partitioner.getOutgoingBatches(id).terminate();
      }

      int remaining = remaingReceiverCount.decrementAndGet();
      if (remaining == 0) {
        done = true;
      }
    }
  }

  @Override
  public void close() throws Exception {
    logger.debug("Partition sender stopping.");
    super.close();

    if (partitioner != null) {
      updateAggregateStats();
      partitioner.clear();
    }

    if (closeIncoming) {
      ((CloseableRecordBatch) incoming).close();
    }
  }

  private void sendEmptyBatch(boolean isLast) {
    BatchSchema schema = incoming.getSchema();
    if (schema == null) {
      // If the incoming batch has no schema (possible when there are no input records),
      // create an empty schema to avoid NPE.
      schema = BatchSchema.newBuilder().build();
    }

    FragmentHandle handle = context.getHandle();
    for (MinorFragmentEndpoint destination : popConfig.getDestinations()) {
      AccountingDataTunnel tunnel = context.getDataTunnel(destination.getEndpoint());
      FragmentWritableBatch writableBatch = FragmentWritableBatch.getEmptyBatchWithSchema(
          isLast,
          handle.getQueryId(),
          handle.getMajorFragmentId(),
          handle.getMinorFragmentId(),
          operator.getOppositeMajorFragmentId(),
          destination.getId(),
          schema);
      stats.startWait();
      try {
        tunnel.sendRecordBatch(writableBatch);
      } finally {
        stats.stopWait();
      }
    }
    stats.addLongStat(Metric.BATCHES_SENT, 1);
  }

  @VisibleForTesting
  protected PartitionerDecorator getPartitioner() {
    return partitioner;
  }
}