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 static ReaderContextBuilder newBuilder(ReaderContext readerContext) { 047 return new ReaderContextBuilder(readerContext); 048 } 049 050 private ReaderContextBuilder(ReaderContext readerContext) { 051 this.filePath = readerContext.getFilePath(); 052 this.fsdis = readerContext.getInputStreamWrapper(); 053 this.fileSize = readerContext.getFileSize(); 054 this.hfs = readerContext.getFileSystem(); 055 this.type = readerContext.getReaderType(); 056 } 057 058 public ReaderContextBuilder withFilePath(Path filePath) { 059 this.filePath = filePath; 060 return this; 061 } 062 063 public ReaderContextBuilder withFileSize(long fileSize) { 064 this.fileSize = fileSize; 065 return this; 066 } 067 068 public ReaderContextBuilder withInputStreamWrapper(FSDataInputStreamWrapper fsdis) { 069 this.fsdis = fsdis; 070 return this; 071 } 072 073 public ReaderContextBuilder withFileSystem(HFileSystem hfs) { 074 this.hfs = hfs; 075 return this; 076 } 077 078 public ReaderContextBuilder withFileSystem(FileSystem fs) { 079 if (!(fs instanceof HFileSystem)) { 080 this.hfs = new HFileSystem(fs); 081 } else { 082 this.hfs = (HFileSystem) fs; 083 } 084 return this; 085 } 086 087 public ReaderContextBuilder withPrimaryReplicaReader(boolean primaryReplicaReader) { 088 this.primaryReplicaReader = primaryReplicaReader; 089 return this; 090 } 091 092 public ReaderContextBuilder withReaderType(ReaderType type) { 093 this.type = type; 094 return this; 095 } 096 097 public ReaderContextBuilder withFileSystemAndPath(FileSystem fs, Path filePath) 098 throws IOException { 099 this.withFileSystem(fs).withFilePath(filePath).withFileSize(fs.getFileStatus(filePath).getLen()) 100 .withInputStreamWrapper(new FSDataInputStreamWrapper(fs, filePath)); 101 return this; 102 } 103 104 public ReaderContext build() { 105 validateFields(); 106 return new ReaderContext(filePath, fsdis, fileSize, hfs, primaryReplicaReader, type); 107 } 108 109 private void validateFields() throws IllegalArgumentException { 110 checkNotNull(filePath, "Illegal ReaderContext, no filePath specified."); 111 checkNotNull(fsdis, "Illegal ReaderContext, no StreamWrapper specified."); 112 checkNotNull(hfs, "Illegal ReaderContext, no HFileSystem specified."); 113 checkArgument(fileSize > 0L, "Illegal ReaderContext, fileSize <= 0"); 114 } 115}