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

import io.netty.buffer.ByteBuf;

import java.util.List;

import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.memory.OutOfMemoryException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.ops.MetricDef;
import org.apache.drill.exec.physical.config.BroadcastSender;
import org.apache.drill.exec.physical.impl.BaseRootExec;
import org.apache.drill.exec.physical.impl.SendingAccountor;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.GeneralRPCProtos;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.record.FragmentWritableBatch;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.WritableBatch;
import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.data.DataTunnel;
import org.apache.drill.exec.work.ErrorHelper;

/**
 * Broadcast Sender broadcasts incoming batches to all receivers (one or more).
 * This is useful in cases such as broadcast join where sending the entire table to join
 * to all nodes is cheaper than merging and computing all the joins in the same node.
 */
public class BroadcastSenderRootExec extends BaseRootExec {
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BroadcastSenderRootExec.class);
  private final FragmentContext context;
  private final BroadcastSender config;
  private final DataTunnel[] tunnels;
  private final ExecProtos.FragmentHandle handle;
  private volatile boolean ok;
  private final RecordBatch incoming;

  public enum Metric implements MetricDef {
    N_RECEIVERS,
    BYTES_SENT;
    @Override
    public int metricId() {
      return ordinal();
    }
  }

  public BroadcastSenderRootExec(FragmentContext context,
                                 RecordBatch incoming,
                                 BroadcastSender config) throws OutOfMemoryException {
    super(context, config);
    this.ok = true;
    this.context = context;
    this.incoming = incoming;
    this.config = config;
    this.handle = context.getHandle();
    List<DrillbitEndpoint> destinations = config.getDestinations();
    this.tunnels = new DataTunnel[destinations.size()];
    for(int i = 0; i < destinations.size(); ++i) {
      FragmentHandle opp = handle.toBuilder().setMajorFragmentId(config.getOppositeMajorFragmentId()).setMinorFragmentId(i).build();
      tunnels[i] = context.getDataTunnel(destinations.get(i), opp);
    }
  }

  @Override
  public boolean innerNext() {
    if(!ok) {
      context.fail(statusHandler.ex);
      return false;
    }

    RecordBatch.IterOutcome out = next(incoming);
    logger.debug("Outcome of sender next {}", out);
    switch(out){
      case STOP:
      case NONE:
        for (int i = 0; i < tunnels.length; ++i) {
          FragmentWritableBatch b2 = FragmentWritableBatch.getEmptyLast(handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), config.getOppositeMajorFragmentId(), i);
          stats.startWait();
          try {
            tunnels[i].sendRecordBatch(this.statusHandler, b2);
          } finally {
            stats.stopWait();
          }
          statusHandler.sendCount.increment();
        }

        return false;

      case OK_NEW_SCHEMA:
      case OK:
        WritableBatch writableBatch = incoming.getWritableBatch();
        if (tunnels.length > 1) {
          writableBatch.retainBuffers(tunnels.length - 1);
        }
        for (int i = 0; i < tunnels.length; ++i) {
          FragmentWritableBatch batch = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), config.getOppositeMajorFragmentId(), i, writableBatch);
          updateStats(batch);
          stats.startWait();
          try {
            tunnels[i].sendRecordBatch(this.statusHandler, batch);
          } finally {
            stats.stopWait();
          }
          statusHandler.sendCount.increment();
        }

        return ok;

      case NOT_YET:
      default:
        throw new IllegalStateException();
    }
  }

  public void updateStats(FragmentWritableBatch writableBatch) {
    stats.setLongStat(Metric.N_RECEIVERS, tunnels.length);
    stats.addLongStat(Metric.BYTES_SENT, writableBatch.getByteCount());
  }

  /*
  private boolean waitAllFutures(boolean haltOnError) {
    for (DrillRpcFuture<?> responseFuture : responseFutures) {
      try {
        GeneralRPCProtos.Ack ack = (GeneralRPCProtos.Ack) responseFuture.checkedGet();
        if(!ack.getOk()) {
          ok = false;
          if (haltOnError) {
            return false;
          }
        }
      } catch (RpcException e) {
        logger.error("Error sending batch to receiver: " + e);
        ok = false;
        if (haltOnError) {
          return false;
        }
      }
    }
    return true;
  }
*/

  @Override
  public void stop() {
      ok = false;
      statusHandler.sendCount.waitForSendComplete();
      oContext.close();
      incoming.cleanup();
  }

  private StatusHandler statusHandler = new StatusHandler();
  private class StatusHandler extends BaseRpcOutcomeListener<GeneralRPCProtos.Ack> {
    volatile RpcException ex;
    private final SendingAccountor sendCount = new SendingAccountor();

    @Override
    public void success(Ack value, ByteBuf buffer) {
      sendCount.decrement();
      super.success(value, buffer);
    }

    @Override
    public void failed(RpcException ex) {
      sendCount.decrement();
      logger.error("Failure while sending data to user.", ex);
      boolean verbose = context.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
      ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger,
        verbose);
      ok = false;
      this.ex = ex;
    }
  }

}