aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
blob: d47e8d9c00fc05ea013534178183cdf13bd1bbc7 (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
/*
 * 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.ops;

import java.security.PrivilegedExceptionAction;
import java.util.concurrent.Callable;

import org.apache.drill.exec.exception.OutOfMemoryException;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.hadoop.security.UserGroupInformation;

import org.apache.drill.shaded.guava.com.google.common.util.concurrent.ListenableFuture;
import org.apache.drill.shaded.guava.com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.drill.shaded.guava.com.google.common.util.concurrent.MoreExecutors;

class OperatorContextImpl extends BaseOperatorContext implements AutoCloseable {
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorContextImpl.class);

  private boolean closed = false;
  private final OperatorStats stats;

  /**
   * This lazily initialized executor service is used to submit a {@link Callable task} that needs a proxy user. There
   * is no pool that is created; this pool is a decorator around {@link WorkManager#executor the worker pool} that
   * returns a {@link ListenableFuture future} for every task that is submitted. For the shutdown sequence,
   * see {@link WorkManager#close}.
   */
  private ListeningExecutorService delegatePool;

  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContextImpl context) throws OutOfMemoryException {
    this(popConfig, context, null);
  }

  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContextImpl context, OperatorStats stats)
      throws OutOfMemoryException {
    super(context,
          context.getNewChildAllocator(popConfig.getClass().getSimpleName(),
              popConfig.getOperatorId(), popConfig.getInitialAllocation(), popConfig.getMaxAllocation()),
          popConfig);
    if (stats != null) {
      this.stats = stats;
    } else {
      OpProfileDef def =
          new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(),
                           OperatorUtilities.getChildCount(popConfig));
      this.stats = context.getStats().newOperatorStats(def, allocator);
    }
  }

  public boolean isClosed() {
    return closed;
  }

  @Override
  public void close() {
    if (closed) {
      logger.debug("Attempted to close Operator context for {}, but context is already closed", popConfig != null ? getName() : null);
      return;
    }
    logger.debug("Closing context for {}", popConfig != null ? getName() : null);
    closed = true;
    super.close();
  }

  @Override
  public OperatorStats getStats() {
    return stats;
  }

  @Override
  public <RESULT> ListenableFuture<RESULT> runCallableAs(final UserGroupInformation proxyUgi,
                                                         final Callable<RESULT> callable) {
    synchronized (this) {
      if (delegatePool == null) {
        delegatePool = MoreExecutors.listeningDecorator(getExecutor());
      }
    }
    return delegatePool.submit(new Callable<RESULT>() {
      @Override
      public RESULT call() throws Exception {
        final Thread currentThread = Thread.currentThread();
        final String originalThreadName = currentThread.getName();
        currentThread.setName(proxyUgi.getUserName() + ":task-delegate-thread");
        final RESULT result;
        try {
          result = proxyUgi.doAs(new PrivilegedExceptionAction<RESULT>() {
            @Override
            public RESULT run() throws Exception {
              return callable.call();
            }
          });
        } finally {
          currentThread.setName(originalThreadName);
        }
        return result;
      }
    });
  }
}