aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
blob: 3ac045f9444f281c041f3f3b2a999370490782ac (plain) (blame)
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
/*
 * 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.spark.network.netty;

import java.io.File;
import java.io.FileInputStream;

import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.DefaultFileRegion;

import org.apache.spark.storage.BlockId;
import org.apache.spark.storage.FileSegment;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

class FileServerHandler extends SimpleChannelInboundHandler<String> {

  private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName());

  private final PathResolver pResolver;

  FileServerHandler(PathResolver pResolver){
    this.pResolver = pResolver;
  }

  @Override
  public void channelRead0(ChannelHandlerContext ctx, String blockIdString) {
    BlockId blockId = BlockId.apply(blockIdString);
    FileSegment fileSegment = pResolver.getBlockLocation(blockId);
    // if getBlockLocation returns null, close the channel
    if (fileSegment == null) {
      //ctx.close();
      return;
    }
    File file = fileSegment.file();
    if (file.exists()) {
      if (!file.isFile()) {
        ctx.write(new FileHeader(0, blockId).buffer());
        ctx.flush();
        return;
      }
      long length = fileSegment.length();
      if (length > Integer.MAX_VALUE || length <= 0) {
        ctx.write(new FileHeader(0, blockId).buffer());
        ctx.flush();
        return;
      }
      int len = (int) length;
      ctx.write((new FileHeader(len, blockId)).buffer());
      try {
        ctx.write(new DefaultFileRegion(new FileInputStream(file)
          .getChannel(), fileSegment.offset(), fileSegment.length()));
      } catch (Exception e) {
          LOG.error("Exception: ", e);
      }
    } else {
      ctx.write(new FileHeader(0, blockId).buffer());
    }
    ctx.flush();
  }

  @Override
  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
    LOG.error("Exception: ", cause);
    ctx.close();
  }
}