HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / ScanInfo.java
blob94cd00bd800e518be3202ba8bcbf6606d30d4e3f
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
19 package org.apache.hadoop.hbase.regionserver;
21 import org.apache.commons.lang3.builder.ToStringBuilder;
22 import org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.hbase.CellComparator;
24 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.KeepDeletedCells;
27 import org.apache.hadoop.hbase.util.Bytes;
28 import org.apache.hadoop.hbase.util.ClassSize;
29 import org.apache.yetus.audience.InterfaceAudience;
31 /**
32 * Immutable information for scans over a store.
34 // Has to be public for PartitionedMobCompactor to access; ditto on tests making use of a few of
35 // the accessors below. Shutdown access. TODO
36 @InterfaceAudience.Private
37 public class ScanInfo {
38 private byte[] family;
39 private int minVersions;
40 private int maxVersions;
41 private long ttl;
42 private KeepDeletedCells keepDeletedCells;
43 private long timeToPurgeDeletes;
44 private CellComparator comparator;
45 private long tableMaxRowSize;
46 private boolean usePread;
47 private long cellsPerTimeoutCheck;
48 private boolean parallelSeekEnabled;
49 private final long preadMaxBytes;
50 private final boolean newVersionBehavior;
52 public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
53 + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
54 + (4 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_BOOLEAN));
56 /**
57 * @param conf
58 * @param family {@link ColumnFamilyDescriptor} describing the column family
59 * @param ttl Store's TTL (in ms)
60 * @param timeToPurgeDeletes duration in ms after which a delete marker can be purged during a
61 * major compaction.
62 * @param comparator The store's comparator
64 public ScanInfo(Configuration conf, ColumnFamilyDescriptor family, long ttl,
65 long timeToPurgeDeletes, CellComparator comparator) {
66 this(conf, family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl,
67 family.getKeepDeletedCells(), family.getBlocksize(), timeToPurgeDeletes, comparator,
68 family.isNewVersionBehavior());
71 private static long getCellsPerTimeoutCheck(Configuration conf) {
72 long perHeartbeat = conf.getLong(StoreScanner.HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK,
73 StoreScanner.DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK);
74 return perHeartbeat > 0 ? perHeartbeat
75 : StoreScanner.DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK;
78 /**
79 * @param conf
80 * @param family Name of this store's column family
81 * @param minVersions Store's MIN_VERSIONS setting
82 * @param maxVersions Store's VERSIONS setting
83 * @param ttl Store's TTL (in ms)
84 * @param blockSize Store's block size
85 * @param timeToPurgeDeletes duration in ms after which a delete marker can
86 * be purged during a major compaction.
87 * @param keepDeletedCells Store's keepDeletedCells setting
88 * @param comparator The store's comparator
90 public ScanInfo(Configuration conf, byte[] family, int minVersions, int maxVersions, long ttl,
91 KeepDeletedCells keepDeletedCells, long blockSize, long timeToPurgeDeletes,
92 CellComparator comparator, boolean newVersionBehavior) {
93 this(family, minVersions, maxVersions, ttl, keepDeletedCells, timeToPurgeDeletes, comparator,
94 conf.getLong(HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT),
95 conf.getBoolean("hbase.storescanner.use.pread", false), getCellsPerTimeoutCheck(conf),
96 conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false),
97 conf.getLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 4 * blockSize), newVersionBehavior);
100 private ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl,
101 KeepDeletedCells keepDeletedCells, long timeToPurgeDeletes, CellComparator comparator,
102 long tableMaxRowSize, boolean usePread, long cellsPerTimeoutCheck,
103 boolean parallelSeekEnabled, long preadMaxBytes, boolean newVersionBehavior) {
104 this.family = family;
105 this.minVersions = minVersions;
106 this.maxVersions = maxVersions;
107 this.ttl = ttl;
108 this.keepDeletedCells = keepDeletedCells;
109 this.timeToPurgeDeletes = timeToPurgeDeletes;
110 this.comparator = comparator;
111 this.tableMaxRowSize = tableMaxRowSize;
112 this.usePread = usePread;
113 this.cellsPerTimeoutCheck = cellsPerTimeoutCheck;
114 this.parallelSeekEnabled = parallelSeekEnabled;
115 this.preadMaxBytes = preadMaxBytes;
116 this.newVersionBehavior = newVersionBehavior;
119 long getTableMaxRowSize() {
120 return this.tableMaxRowSize;
123 boolean isUsePread() {
124 return this.usePread;
127 long getCellsPerTimeoutCheck() {
128 return this.cellsPerTimeoutCheck;
131 boolean isParallelSeekEnabled() {
132 return this.parallelSeekEnabled;
135 public byte[] getFamily() {
136 return family;
139 public int getMinVersions() {
140 return minVersions;
143 public int getMaxVersions() {
144 return maxVersions;
147 public long getTtl() {
148 return ttl;
151 public KeepDeletedCells getKeepDeletedCells() {
152 return keepDeletedCells;
155 public long getTimeToPurgeDeletes() {
156 return timeToPurgeDeletes;
159 public CellComparator getComparator() {
160 return comparator;
163 long getPreadMaxBytes() {
164 return preadMaxBytes;
167 public boolean isNewVersionBehavior() {
168 return newVersionBehavior;
172 * Used by CP users for customizing max versions, ttl and keepDeletedCells.
174 ScanInfo customize(int maxVersions, long ttl, KeepDeletedCells keepDeletedCells) {
175 return customize(maxVersions, ttl, keepDeletedCells, minVersions, timeToPurgeDeletes);
179 * Used by CP users for customizing max versions, ttl, keepDeletedCells, min versions,
180 * and time to purge deletes.
182 ScanInfo customize(int maxVersions, long ttl, KeepDeletedCells keepDeletedCells,
183 int minVersions, long timeToPurgeDeletes) {
184 return new ScanInfo(family, minVersions, maxVersions, ttl, keepDeletedCells, timeToPurgeDeletes,
185 comparator, tableMaxRowSize, usePread, cellsPerTimeoutCheck, parallelSeekEnabled,
186 preadMaxBytes, newVersionBehavior);
189 @Override
190 public String toString() {
191 return new ToStringBuilder(this)
192 .append("family", Bytes.toStringBinary(family))
193 .append("minVersions", minVersions)
194 .append("maxVersions", maxVersions)
195 .append("ttl", ttl)
196 .append("keepDeletedCells", keepDeletedCells)
197 .append("timeToPurgeDeletes", timeToPurgeDeletes)
198 .append("tableMaxRowSize", tableMaxRowSize)
199 .append("usePread", usePread)
200 .append("cellsPerTimeoutCheck", cellsPerTimeoutCheck)
201 .append("parallelSeekEnabled", parallelSeekEnabled)
202 .append("preadMaxBytes", preadMaxBytes)
203 .append("newVersionBehavior", newVersionBehavior).toString();