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