aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java
blob: 91d3647513c8dc4d54cbc116af9b984afbad9646 (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
/**
 * 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.producer;

import org.apache.drill.common.types.TypeProtos.MajorType;
import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.expr.TypeHelper;
import org.apache.drill.exec.memory.OutOfMemoryException;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.config.ProducerConsumer;
import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
import org.apache.drill.exec.record.AbstractRecordBatch;
import org.apache.drill.exec.record.BatchSchema;
import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
import org.apache.drill.exec.record.MaterializedField;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.RecordBatchLoader;
import org.apache.drill.exec.record.TransferPair;
import org.apache.drill.exec.record.VectorContainer;
import org.apache.drill.exec.record.VectorWrapper;
import org.apache.drill.exec.record.WritableBatch;
import org.apache.drill.exec.vector.ValueVector;

import java.util.Queue;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingDeque;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingDeque;

public class ProducerConsumerBatch extends AbstractRecordBatch {
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProducerConsumerBatch.class);

  private RecordBatch incoming;
  private Thread producer = new Thread(new Producer(), Thread.currentThread().getName() + " - Producer Thread");
  private boolean running = false;
  private BlockingDeque<RecordBatchDataWrapper> queue;
  private int recordCount;
  private BatchSchema schema;
  private boolean stop = false;
  private final CountDownLatch cleanUpLatch = new CountDownLatch(1); // used to wait producer to clean up

  protected ProducerConsumerBatch(ProducerConsumer popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException {
    super(popConfig, context);
    this.incoming = incoming;
    this.queue = new LinkedBlockingDeque<>(popConfig.getSize());
  }

  @Override
  public IterOutcome innerNext() {
    if (!running) {
      producer.start();
      running = true;
    }
    RecordBatchDataWrapper wrapper;
    try {
      stats.startWait();
      wrapper = queue.take();
      logger.debug("Got batch from queue");
    } catch (InterruptedException e) {
      if (!(context.isCancelled() || context.isFailed())) {
        context.fail(e);
      }
      return IterOutcome.STOP;
    } finally {
      stats.stopWait();
    }
    if (wrapper.finished) {
      return IterOutcome.NONE;
    } else if (wrapper.failed) {
      return IterOutcome.STOP;
    }

    recordCount = wrapper.batch.getRecordCount();
    boolean newSchema = load(wrapper.batch);

    return newSchema ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK;
  }

  private boolean load(RecordBatchData batch) {
    VectorContainer newContainer = batch.getContainer();
    if (schema != null && newContainer.getSchema().equals(schema)) {
      container.zeroVectors();
      BatchSchema schema = container.getSchema();
      for (int i = 0; i < container.getNumberOfColumns(); i++) {
        MaterializedField field = schema.getColumn(i);
        MajorType type = field.getType();
        ValueVector vOut = container.getValueAccessorById(TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()),
                container.getValueVectorId(field.getPath()).getFieldIds()).getValueVector();
        ValueVector vIn = newContainer.getValueAccessorById(TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()),
                newContainer.getValueVectorId(field.getPath()).getFieldIds()).getValueVector();
        TransferPair tp = vIn.makeTransferPair(vOut);
        tp.transfer();
      }
      return false;
    } else {
      container.clear();
      for (VectorWrapper w : newContainer) {
        container.add(w.getValueVector());
      }
      container.buildSchema(SelectionVectorMode.NONE);
      schema = container.getSchema();
      return true;
    }
  }

  private class Producer implements Runnable {

    RecordBatchDataWrapper wrapper;

    @Override
    public void run() {
      try {
        if (stop) return;
        outer:
        while (true) {
          IterOutcome upstream = incoming.next();
          switch (upstream) {
            case NONE:
              break outer;
            case STOP:
              queue.putFirst(new RecordBatchDataWrapper(null, false, true));
              return;
            case OK_NEW_SCHEMA:
            case OK:
              try {
                if (!stop) {
                  wrapper = new RecordBatchDataWrapper(new RecordBatchData(incoming), false, false);
                  queue.put(wrapper);
                }
              } catch (InterruptedException e) {
                wrapper.batch.getContainer().zeroVectors();
                throw e;
              }
              break;
            default:
              throw new UnsupportedOperationException();
          }
        }

        queue.put(new RecordBatchDataWrapper(null, true, false));
      } catch (InterruptedException e) {
        if (!(context.isCancelled() || context.isFailed())) {
          context.fail(e);
        }
      } finally {
        cleanUpLatch.countDown();
      }
    }
  }

  private void clearQueue() {
    RecordBatchDataWrapper wrapper;
    while ((wrapper = queue.poll()) != null) {
      if (wrapper.batch != null) {
        wrapper.batch.getContainer().clear();
      }
    }
  }

  @Override
  protected void killIncoming(boolean sendUpstream) {
    stop = true;
    producer.interrupt();
    try {
      producer.join();
    } catch (InterruptedException e) {
      logger.warn("Interrupted while waiting for producer thread");
    }
  }

  @Override
  public void cleanup() {
    stop = true;
    try {
      cleanUpLatch.await();
    } catch (InterruptedException e) {
      logger.warn("Interrupted while waiting for producer to clean up first. I will try to clean up now...", e);
    } finally {
      super.cleanup();
      clearQueue();
      incoming.cleanup();
    }
  }

  @Override
  public int getRecordCount() {
    return recordCount;
  }

  private static class RecordBatchDataWrapper {
    RecordBatchData batch;
    boolean finished;
    boolean failed;

    RecordBatchDataWrapper(RecordBatchData batch, boolean finished, boolean failed) {
      this.batch = batch;
      this.finished = finished;
      this.failed = failed;
    }
  }
}