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.regionserver;
019
020import java.net.InetSocketAddress;
021import java.util.Collection;
022import java.util.function.Supplier;
023import org.apache.hadoop.fs.Path;
024import org.apache.hadoop.hbase.CellComparator;
025import org.apache.hadoop.hbase.TableName;
026import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
027import org.apache.hadoop.hbase.client.RegionInfo;
028import org.apache.hadoop.hbase.io.HeapSize;
029import org.apache.hadoop.hbase.io.crypto.Encryption;
030import org.apache.hadoop.hbase.io.hfile.CacheConfig;
031import org.apache.hadoop.hbase.util.ClassSize;
032import org.apache.yetus.audience.InterfaceAudience;
033
034/**
035 * This carries the immutable information and references on some of the meta data about the HStore.
036 * This meta data can be used across the HFileWriter/Readers and other HStore consumers without the
037 * need of passing around the complete store.
038 */
039@InterfaceAudience.Private
040public final class StoreContext implements HeapSize {
041  public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HStore.class, false);
042
043  private final int blockSize;
044  private final Encryption.Context encryptionContext;
045  private final CacheConfig cacheConf;
046  private final HRegionFileSystem regionFileSystem;
047  private final CellComparator comparator;
048  private final BloomType bloomFilterType;
049  private final Supplier<Collection<HStoreFile>> compactedFilesSupplier;
050  private final Supplier<InetSocketAddress[]> favoredNodesSupplier;
051  private final ColumnFamilyDescriptor family;
052  private final Path familyStoreDirectoryPath;
053  private final RegionCoprocessorHost coprocessorHost;
054
055  private StoreContext(Builder builder) {
056    this.blockSize = builder.blockSize;
057    this.encryptionContext = builder.encryptionContext;
058    this.cacheConf = builder.cacheConf;
059    this.regionFileSystem = builder.regionFileSystem;
060    this.comparator = builder.comparator;
061    this.bloomFilterType = builder.bloomFilterType;
062    this.compactedFilesSupplier = builder.compactedFilesSupplier;
063    this.favoredNodesSupplier = builder.favoredNodesSupplier;
064    this.family = builder.family;
065    this.familyStoreDirectoryPath = builder.familyStoreDirectoryPath;
066    this.coprocessorHost = builder.coprocessorHost;
067  }
068
069  public int getBlockSize() {
070    return blockSize;
071  }
072
073  public Encryption.Context getEncryptionContext() {
074    return encryptionContext;
075  }
076
077  public CacheConfig getCacheConf() {
078    return cacheConf;
079  }
080
081  public HRegionFileSystem getRegionFileSystem() {
082    return regionFileSystem;
083  }
084
085  public CellComparator getComparator() {
086    return comparator;
087  }
088
089  public BloomType getBloomFilterType() {
090    return bloomFilterType;
091  }
092
093  public Supplier<Collection<HStoreFile>> getCompactedFilesSupplier() {
094    return compactedFilesSupplier;
095  }
096
097  public InetSocketAddress[] getFavoredNodes() {
098    return favoredNodesSupplier.get();
099  }
100
101  public ColumnFamilyDescriptor getFamily() {
102    return family;
103  }
104
105  public Path getFamilyStoreDirectoryPath() {
106    return familyStoreDirectoryPath;
107  }
108
109  public RegionCoprocessorHost getCoprocessorHost() {
110    return coprocessorHost;
111  }
112
113  public TableName getTableName() {
114    return getRegionInfo().getTable();
115  }
116
117  public RegionInfo getRegionInfo() {
118    return regionFileSystem.getRegionInfo();
119  }
120
121  public boolean isPrimaryReplicaStore() {
122    return getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID;
123  }
124
125  public static Builder getBuilder() {
126    return new Builder();
127  }
128
129  @Override
130  public long heapSize() {
131    return FIXED_OVERHEAD;
132  }
133
134  public static class Builder {
135    private int blockSize;
136    private Encryption.Context encryptionContext;
137    private CacheConfig cacheConf;
138    private HRegionFileSystem regionFileSystem;
139    private CellComparator comparator;
140    private BloomType bloomFilterType;
141    private Supplier<Collection<HStoreFile>> compactedFilesSupplier;
142    private Supplier<InetSocketAddress[]> favoredNodesSupplier;
143    private ColumnFamilyDescriptor family;
144    private Path familyStoreDirectoryPath;
145    private RegionCoprocessorHost coprocessorHost;
146
147    public Builder withBlockSize(int blockSize) {
148      this.blockSize = blockSize;
149      return this;
150    }
151
152    public Builder withEncryptionContext(Encryption.Context encryptionContext) {
153      this.encryptionContext = encryptionContext;
154      return this;
155    }
156
157    public Builder withCacheConfig(CacheConfig cacheConf) {
158      this.cacheConf = cacheConf;
159      return this;
160    }
161
162    public Builder withRegionFileSystem(HRegionFileSystem regionFileSystem) {
163      this.regionFileSystem = regionFileSystem;
164      return this;
165    }
166
167    public Builder withCellComparator(CellComparator comparator) {
168      this.comparator = comparator;
169      return this;
170    }
171
172    public Builder withBloomType(BloomType bloomFilterType) {
173      this.bloomFilterType = bloomFilterType;
174      return this;
175    }
176
177    public Builder
178      withCompactedFilesSupplier(Supplier<Collection<HStoreFile>> compactedFilesSupplier) {
179      this.compactedFilesSupplier = compactedFilesSupplier;
180      return this;
181    }
182
183    public Builder withFavoredNodesSupplier(Supplier<InetSocketAddress[]> favoredNodesSupplier) {
184      this.favoredNodesSupplier = favoredNodesSupplier;
185      return this;
186    }
187
188    public Builder withColumnFamilyDescriptor(ColumnFamilyDescriptor family) {
189      this.family = family;
190      return this;
191    }
192
193    public Builder withFamilyStoreDirectoryPath(Path familyStoreDirectoryPath) {
194      this.familyStoreDirectoryPath = familyStoreDirectoryPath;
195      return this;
196    }
197
198    public Builder withRegionCoprocessorHost(RegionCoprocessorHost coprocessorHost) {
199      this.coprocessorHost = coprocessorHost;
200      return this;
201    }
202
203    public StoreContext build() {
204      return new StoreContext(this);
205    }
206  }
207
208}