HBASE-26412 Handle sink failure in RegionReplicationSink (#3815)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / ScanInfo.java
blob831ce01f87143363231e3f16e919454a70e42f7e
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.hadoop.conf.Configuration;
22 import org.apache.hadoop.hbase.CellComparator;
23 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
24 import org.apache.hadoop.hbase.HConstants;
25 import org.apache.hadoop.hbase.KeepDeletedCells;
26 import org.apache.hadoop.hbase.util.Bytes;
27 import org.apache.hadoop.hbase.util.ClassSize;
28 import org.apache.yetus.audience.InterfaceAudience;
30 /**
31 * Immutable information for scans over a store.
33 // Has to be public for PartitionedMobCompactor to access; ditto on tests making use of a few of
34 // the accessors below. Shutdown access. TODO
35 @InterfaceAudience.Private
36 public class ScanInfo {
37 private byte[] family;
38 private int minVersions;
39 private int maxVersions;
40 private long ttl;
41 private KeepDeletedCells keepDeletedCells;
42 private long timeToPurgeDeletes;
43 private CellComparator comparator;
44 private long tableMaxRowSize;
45 private boolean usePread;
46 private long cellsPerTimeoutCheck;
47 private boolean parallelSeekEnabled;
48 private final long preadMaxBytes;
49 private final boolean newVersionBehavior;
51 public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
52 + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
53 + (4 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_BOOLEAN));
55 /**
56 * @param conf
57 * @param family {@link ColumnFamilyDescriptor} describing the column family
58 * @param ttl Store's TTL (in ms)
59 * @param timeToPurgeDeletes duration in ms after which a delete marker can be purged during a
60 * major compaction.
61 * @param comparator The store's comparator
63 public ScanInfo(Configuration conf, ColumnFamilyDescriptor family, long ttl,
64 long timeToPurgeDeletes, CellComparator comparator) {
65 this(conf, family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl,
66 family.getKeepDeletedCells(), family.getBlocksize(), timeToPurgeDeletes, comparator,
67 family.isNewVersionBehavior());
70 private static long getCellsPerTimeoutCheck(Configuration conf) {
71 long perHeartbeat = conf.getLong(StoreScanner.HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK,
72 StoreScanner.DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK);
73 return perHeartbeat > 0 ? perHeartbeat
74 : StoreScanner.DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK;
77 /**
78 * @param conf
79 * @param family Name of this store's column family
80 * @param minVersions Store's MIN_VERSIONS setting
81 * @param maxVersions Store's VERSIONS setting
82 * @param ttl Store's TTL (in ms)
83 * @param blockSize Store's block size
84 * @param timeToPurgeDeletes duration in ms after which a delete marker can
85 * be purged during a major compaction.
86 * @param keepDeletedCells Store's keepDeletedCells setting
87 * @param comparator The store's comparator
89 public ScanInfo(Configuration conf, byte[] family, int minVersions, int maxVersions, long ttl,
90 KeepDeletedCells keepDeletedCells, long blockSize, long timeToPurgeDeletes,
91 CellComparator comparator, boolean newVersionBehavior) {
92 this(family, minVersions, maxVersions, ttl, keepDeletedCells, timeToPurgeDeletes, comparator,
93 conf.getLong(HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT),
94 conf.getBoolean("hbase.storescanner.use.pread", false), getCellsPerTimeoutCheck(conf),
95 conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false),
96 conf.getLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 4 * blockSize), newVersionBehavior);
99 private ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl,
100 KeepDeletedCells keepDeletedCells, long timeToPurgeDeletes, CellComparator comparator,
101 long tableMaxRowSize, boolean usePread, long cellsPerTimeoutCheck,
102 boolean parallelSeekEnabled, long preadMaxBytes, boolean newVersionBehavior) {
103 this.family = family;
104 this.minVersions = minVersions;
105 this.maxVersions = maxVersions;
106 this.ttl = ttl;
107 this.keepDeletedCells = keepDeletedCells;
108 this.timeToPurgeDeletes = timeToPurgeDeletes;
109 this.comparator = comparator;
110 this.tableMaxRowSize = tableMaxRowSize;
111 this.usePread = usePread;
112 this.cellsPerTimeoutCheck = cellsPerTimeoutCheck;
113 this.parallelSeekEnabled = parallelSeekEnabled;
114 this.preadMaxBytes = preadMaxBytes;
115 this.newVersionBehavior = newVersionBehavior;
118 long getTableMaxRowSize() {
119 return this.tableMaxRowSize;
122 boolean isUsePread() {
123 return this.usePread;
126 long getCellsPerTimeoutCheck() {
127 return this.cellsPerTimeoutCheck;
130 boolean isParallelSeekEnabled() {
131 return this.parallelSeekEnabled;
134 public byte[] getFamily() {
135 return family;
138 public int getMinVersions() {
139 return minVersions;
142 public int getMaxVersions() {
143 return maxVersions;
146 public long getTtl() {
147 return ttl;
150 public KeepDeletedCells getKeepDeletedCells() {
151 return keepDeletedCells;
154 public long getTimeToPurgeDeletes() {
155 return timeToPurgeDeletes;
158 public CellComparator getComparator() {
159 return comparator;
162 long getPreadMaxBytes() {
163 return preadMaxBytes;
166 public boolean isNewVersionBehavior() {
167 return newVersionBehavior;
171 * Used by CP users for customizing max versions, ttl and keepDeletedCells.
173 ScanInfo customize(int maxVersions, long ttl, KeepDeletedCells keepDeletedCells) {
174 return customize(maxVersions, ttl, keepDeletedCells, minVersions, timeToPurgeDeletes);
178 * Used by CP users for customizing max versions, ttl, keepDeletedCells, min versions,
179 * and time to purge deletes.
181 ScanInfo customize(int maxVersions, long ttl, KeepDeletedCells keepDeletedCells,
182 int minVersions, long timeToPurgeDeletes) {
183 return new ScanInfo(family, minVersions, maxVersions, ttl, keepDeletedCells, timeToPurgeDeletes,
184 comparator, tableMaxRowSize, usePread, cellsPerTimeoutCheck, parallelSeekEnabled,
185 preadMaxBytes, newVersionBehavior);