001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.io.hfile;
020
021import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
022import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
023import java.io.IOException;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.fs.HFileSystem;
027import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
028import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
029import org.apache.yetus.audience.InterfaceAudience;
030
031/**
032 * A builder that helps in building up the ReaderContext
033 */
034@InterfaceAudience.Private
035public class ReaderContextBuilder {
036  private Path filePath;
037  private FSDataInputStreamWrapper fsdis;
038  private long fileSize;
039  private HFileSystem hfs;
040  private boolean primaryReplicaReader = true;
041  private ReaderType type = ReaderType.PREAD;
042
043  public ReaderContextBuilder() {}
044
045  public ReaderContextBuilder withFilePath(Path filePath) {
046    this.filePath = filePath;
047    return this;
048  }
049
050  public ReaderContextBuilder withFileSize(long fileSize) {
051    this.fileSize = fileSize;
052    return this;
053  }
054
055  public ReaderContextBuilder withInputStreamWrapper(FSDataInputStreamWrapper fsdis) {
056    this.fsdis = fsdis;
057    return this;
058  }
059
060  public ReaderContextBuilder withFileSystem(HFileSystem hfs) {
061    this.hfs = hfs;
062    return this;
063  }
064
065  public ReaderContextBuilder withFileSystem(FileSystem fs) {
066    if (!(fs instanceof HFileSystem)) {
067      this.hfs = new HFileSystem(fs);
068    } else {
069      this.hfs = (HFileSystem) fs;
070    }
071    return this;
072  }
073
074  public ReaderContextBuilder withPrimaryReplicaReader(boolean primaryReplicaReader) {
075    this.primaryReplicaReader = primaryReplicaReader;
076    return this;
077  }
078
079  public ReaderContextBuilder withReaderType(ReaderType type) {
080    this.type = type;
081    return this;
082  }
083
084  public ReaderContextBuilder withFileSystemAndPath(FileSystem fs, Path filePath)
085      throws IOException {
086    this.withFileSystem(fs)
087        .withFilePath(filePath)
088        .withFileSize(fs.getFileStatus(filePath).getLen())
089        .withInputStreamWrapper(new FSDataInputStreamWrapper(fs, filePath));
090    return this;
091  }
092
093  public ReaderContext build() {
094    validateFields();
095    return new ReaderContext(filePath, fsdis, fileSize, hfs, primaryReplicaReader, type);
096  }
097
098  private void validateFields() throws IllegalArgumentException {
099    checkNotNull(filePath, "Illegal ReaderContext, no filePath specified.");
100    checkNotNull(fsdis, "Illegal ReaderContext, no StreamWrapper specified.");
101    checkNotNull(hfs, "Illegal ReaderContext, no HFileSystem specified.");
102    checkArgument(fileSize > 0L, "Illegal ReaderContext, fileSize <= 0");
103  }
104}