aboutsummaryrefslogtreecommitdiff
path: root/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
blob: 6bc7bb02a93303094cd6d975ecfa714e83018de3 (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
/*
 * 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.store.schedule;


import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.TimeUnit;

import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.drill.exec.metrics.DrillMetrics;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.store.TimedCallable;
import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodecFactory;

import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableRangeMap;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
import org.apache.drill.shaded.guava.com.google.common.collect.Range;

import static org.apache.commons.lang3.builder.ToStringStyle.SHORT_PREFIX_STYLE;

public class BlockMapBuilder {
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BlockMapBuilder.class);
  static final MetricRegistry metrics = DrillMetrics.getRegistry();
  static final String BLOCK_MAP_BUILDER_TIMER = MetricRegistry.name(BlockMapBuilder.class, "blockMapBuilderTimer");

  private final Map<Path,ImmutableRangeMap<Long,BlockLocation>> blockMapMap = Maps.newConcurrentMap();
  private final FileSystem fs;
  private final ImmutableMap<String,DrillbitEndpoint> endPointMap;
  private final CompressionCodecFactory codecFactory;

  public BlockMapBuilder(FileSystem fs, Collection<DrillbitEndpoint> endpoints) {
    this.fs = fs;
    this.codecFactory = new CompressionCodecFactory(fs.getConf());
    this.endPointMap = buildEndpointMap(endpoints);
  }

  private boolean compressed(FileStatus fileStatus) {
    return codecFactory.getCodec(fileStatus.getPath()) != null;
  }

  public List<CompleteFileWork> generateFileWork(List<FileStatus> files, boolean blockify) throws IOException {

    List<TimedCallable<List<CompleteFileWork>>> readers = new ArrayList<>(files.size());
    for(FileStatus status : files){
      readers.add(new BlockMapReader(status, blockify));
    }
    List<List<CompleteFileWork>> work = TimedCallable.run("Get block maps", logger, readers, 16);
    List<CompleteFileWork> singleList = Lists.newArrayList();
    for(List<CompleteFileWork> innerWorkList : work){
      singleList.addAll(innerWorkList);
    }

    return singleList;

  }

  private class BlockMapReader extends TimedCallable<List<CompleteFileWork>> {
    final FileStatus status;

    // This variable blockify indicates if a single file can be read by multiple threads
    // For examples, for CSV, it is set as true
    // because each row in a CSV file can be considered as an independent record;
    // for json, it is set as false
    // because each row in a json file cannot be determined as a record or not simply by that row alone
    final boolean blockify;

    public BlockMapReader(FileStatus status, boolean blockify) {
      super();
      this.status = status;
      this.blockify = blockify;
    }


    @Override
    protected List<CompleteFileWork> runInner() throws Exception {
      final List<CompleteFileWork> work = new ArrayList<>();

      final Set<String> noDrillbitHosts = logger.isDebugEnabled() ? new HashSet<>() : null;

      boolean error = false;
      if (blockify && !compressed(status)) {
        try {
          ImmutableRangeMap<Long, BlockLocation> rangeMap = getBlockMap(status);
          for (Entry<Range<Long>, BlockLocation> l : rangeMap.asMapOfRanges().entrySet()) {
            work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)),
                l.getValue().getOffset(), l.getValue().getLength(), status.getPath()));
          }
        } catch (IOException e) {
          logger.warn("failure while generating file work.", e);
          error = true;
        }
      }


      if (!blockify || error || compressed(status)) {
        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0,
            status.getLen(), status.getPath()));
      }

      // This if-condition is specific for empty CSV file
      // For CSV files, the global variable blockify is set as true
      // And if this CSV file is empty, rangeMap would be empty also
      // Therefore, at the point before this if-condition, work would not be populated
      if(work.isEmpty()) {
        work.add(new CompleteFileWork(getEndpointByteMap(noDrillbitHosts, new FileStatusWork(status)), 0, 0,
            status.getPath()));
      }

      if (noDrillbitHosts != null) {
        for (String host: noDrillbitHosts) {
          logger.debug("Failure finding Drillbit running on host {}. Skipping affinity to that host.", host);
        }
      }

      return work;
    }

    @Override
    public String toString() {
      return new ToStringBuilder(this, SHORT_PREFIX_STYLE).append("path", status.getPath()).toString();
    }
  }

  private class FileStatusWork implements FileWork{
    private FileStatus status;

    public FileStatusWork(FileStatus status) {
      Preconditions.checkArgument(!status.isDir(), "FileStatus work only works with files, not directories.");
      this.status = status;
    }

    @Override
    public Path getPath() {
      return status.getPath();
    }

    @Override
    public long getStart() {
      return 0;
    }

    @Override
    public long getLength() {
      return status.getLen();
    }



  }

  private ImmutableRangeMap<Long,BlockLocation> buildBlockMap(Path path) throws IOException {
    FileStatus status = fs.getFileStatus(path);
    return buildBlockMap(status);
  }

  /**
   * Builds a mapping of block locations to file byte range
   */
  private ImmutableRangeMap<Long,BlockLocation> buildBlockMap(FileStatus status) throws IOException {
    final Timer.Context context = metrics.timer(BLOCK_MAP_BUILDER_TIMER).time();
    BlockLocation[] blocks;
    ImmutableRangeMap<Long,BlockLocation> blockMap;
    blocks = fs.getFileBlockLocations(status, 0, status.getLen());
    ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new ImmutableRangeMap.Builder<Long,BlockLocation>();
    for (BlockLocation block : blocks) {
      long start = block.getOffset();
      long end = start + block.getLength();
      Range<Long> range = Range.closedOpen(start, end);
      blockMapBuilder = blockMapBuilder.put(range, block);
    }
    blockMap = blockMapBuilder.build();
    blockMapMap.put(status.getPath(), blockMap);
    context.stop();
    return blockMap;
  }

  private ImmutableRangeMap<Long,BlockLocation> getBlockMap(Path path) throws IOException{
    ImmutableRangeMap<Long,BlockLocation> blockMap  = blockMapMap.get(path);
    if(blockMap == null) {
      blockMap = buildBlockMap(path);
    }
    return blockMap;
  }

  private ImmutableRangeMap<Long,BlockLocation> getBlockMap(FileStatus status) throws IOException{
    ImmutableRangeMap<Long,BlockLocation> blockMap  = blockMapMap.get(status.getPath());
    if (blockMap == null) {
      blockMap = buildBlockMap(status);
    }
    return blockMap;
  }


  /**
   * For a given FileWork, calculate how many bytes are available on each on drillbit endpoint
   *
   * @param work the FileWork to calculate endpoint bytes for
   * @throws IOException
   */
  public EndpointByteMap getEndpointByteMap(Set<String> noDrillbitHosts, FileWork work) throws IOException {
    Stopwatch watch = Stopwatch.createStarted();
    Path fileName = work.getPath();


    ImmutableRangeMap<Long, BlockLocation> blockMap = getBlockMap(fileName);
    EndpointByteMapImpl endpointByteMap = new EndpointByteMapImpl();
    long start = work.getStart();
    long end = start + work.getLength();
    Range<Long> rowGroupRange = Range.closedOpen(start, end);

    // Find submap of ranges that intersect with the rowGroup
    ImmutableRangeMap<Long, BlockLocation> subRangeMap = blockMap.subRangeMap(rowGroupRange);

    // Iterate through each block in this submap and get the host for the block location
    for (Map.Entry<Range<Long>, BlockLocation> block : subRangeMap.asMapOfRanges().entrySet()) {
      String[] hosts;
      Range<Long> blockRange = block.getKey();
      try {
        hosts = block.getValue().getHosts();
      } catch (IOException ioe) {
        throw new RuntimeException("Failed to get hosts for block location", ioe);
      }
      Range<Long> intersection = rowGroupRange.intersection(blockRange);
      long bytes = intersection.upperEndpoint() - intersection.lowerEndpoint();

      // For each host in the current block location, add the intersecting bytes to the corresponding endpoint
      for (String host : hosts) {
        DrillbitEndpoint endpoint = getDrillBitEndpoint(host);
        if (endpoint != null) {
          endpointByteMap.add(endpoint, bytes);
        } else if (noDrillbitHosts != null) {
          noDrillbitHosts.add(host);
        }
      }
    }

    logger.debug("FileWork group ({},{}) max bytes {}", work.getPath(), work.getStart(), endpointByteMap.getMaxBytes());

    logger.debug("Took {} ms to set endpoint bytes", watch.stop().elapsed(TimeUnit.MILLISECONDS));
    return endpointByteMap;
  }

  private DrillbitEndpoint getDrillBitEndpoint(String hostName) {
    return endPointMap.get(hostName);
  }

  /**
   * Builds a mapping of Drillbit endpoints to hostnames
   */
  private static ImmutableMap<String, DrillbitEndpoint> buildEndpointMap(Collection<DrillbitEndpoint> endpoints) {
    Stopwatch watch = Stopwatch.createStarted();
    HashMap<String, DrillbitEndpoint> endpointMap = Maps.newHashMap();
    for (DrillbitEndpoint d : endpoints) {
      String hostName = d.getAddress();
      endpointMap.put(hostName, d);
    }
    watch.stop();
    logger.debug("Took {} ms to build endpoint map", watch.elapsed(TimeUnit.MILLISECONDS));
    return ImmutableMap.copyOf(endpointMap);
  }
}