HBASE-21843 RegionGroupingProvider breaks the meta wal file name pattern which may...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / wal / DisabledWALProvider.java
blob75439fe6c523434da7629e6ea4c1680cc3b5fa0f
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.
18 package org.apache.hadoop.hbase.wal;
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.Map;
24 import java.util.OptionalLong;
25 import java.util.Set;
26 import java.util.concurrent.CopyOnWriteArrayList;
27 import java.util.concurrent.atomic.AtomicBoolean;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.hbase.Cell;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.PrivateCellUtil;
34 import org.apache.hadoop.hbase.client.RegionInfo;
35 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
36 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
37 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
38 import org.apache.hadoop.hbase.util.FSUtils;
39 import org.apache.yetus.audience.InterfaceAudience;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
43 // imports for things that haven't moved from regionserver.wal yet.
45 /**
46 * No-op implementation of {@link WALProvider} used when the WAL is disabled.
48 * Should only be used when severe data loss is acceptable.
51 @InterfaceAudience.Private
52 class DisabledWALProvider implements WALProvider {
54 private static final Logger LOG = LoggerFactory.getLogger(DisabledWALProvider.class);
56 WAL disabled;
58 @Override
59 public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
60 if (null != disabled) {
61 throw new IllegalStateException("WALProvider.init should only be called once.");
63 if (null == providerId) {
64 providerId = "defaultDisabled";
66 disabled = new DisabledWAL(new Path(FSUtils.getWALRootDir(conf), providerId), conf, null);
69 @Override
70 public List<WAL> getWALs() {
71 List<WAL> wals = new ArrayList<>(1);
72 wals.add(disabled);
73 return wals;
76 @Override
77 public WAL getWAL(RegionInfo region) throws IOException {
78 return disabled;
81 @Override
82 public void close() throws IOException {
83 disabled.close();
86 @Override
87 public void shutdown() throws IOException {
88 disabled.shutdown();
91 private static class DisabledWAL implements WAL {
92 protected final List<WALActionsListener> listeners = new CopyOnWriteArrayList<>();
93 protected final Path path;
94 protected final WALCoprocessorHost coprocessorHost;
95 protected final AtomicBoolean closed = new AtomicBoolean(false);
97 public DisabledWAL(final Path path, final Configuration conf,
98 final List<WALActionsListener> listeners) {
99 this.coprocessorHost = new WALCoprocessorHost(this, conf);
100 this.path = path;
101 if (null != listeners) {
102 for(WALActionsListener listener : listeners) {
103 registerWALActionsListener(listener);
108 @Override
109 public void registerWALActionsListener(final WALActionsListener listener) {
110 listeners.add(listener);
113 @Override
114 public boolean unregisterWALActionsListener(final WALActionsListener listener) {
115 return listeners.remove(listener);
118 @Override
119 public byte[][] rollWriter() {
120 if (!listeners.isEmpty()) {
121 for (WALActionsListener listener : listeners) {
122 listener.logRollRequested(false);
124 for (WALActionsListener listener : listeners) {
125 try {
126 listener.preLogRoll(path, path);
127 } catch (IOException exception) {
128 LOG.debug("Ignoring exception from listener.", exception);
131 for (WALActionsListener listener : listeners) {
132 try {
133 listener.postLogRoll(path, path);
134 } catch (IOException exception) {
135 LOG.debug("Ignoring exception from listener.", exception);
139 return null;
142 @Override
143 public byte[][] rollWriter(boolean force) {
144 return rollWriter();
147 @Override
148 public void shutdown() {
149 if(closed.compareAndSet(false, true)) {
150 if (!this.listeners.isEmpty()) {
151 for (WALActionsListener listener : this.listeners) {
152 listener.logCloseRequested();
158 @Override
159 public void close() {
160 shutdown();
163 @Override
164 public long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
165 throws IOException {
166 WriteEntry writeEntry = key.getMvcc().begin();
167 if (!edits.isReplay()) {
168 for (Cell cell : edits.getCells()) {
169 PrivateCellUtil.setSequenceId(cell, writeEntry.getWriteNumber());
172 key.setWriteEntry(writeEntry);
173 if (!this.listeners.isEmpty()) {
174 final long start = System.nanoTime();
175 long len = 0;
176 for (Cell cell : edits.getCells()) {
177 len += PrivateCellUtil.estimatedSerializedSizeOf(cell);
179 final long elapsed = (System.nanoTime() - start) / 1000000L;
180 for (WALActionsListener listener : this.listeners) {
181 listener.postAppend(len, elapsed, key, edits);
184 return -1;
187 @Override
188 public void updateStore(byte[] encodedRegionName, byte[] familyName,
189 Long sequenceid, boolean onlyIfGreater) { return; }
191 @Override
192 public void sync() {
193 if (!this.listeners.isEmpty()) {
194 for (WALActionsListener listener : this.listeners) {
195 listener.postSync(0L, 0);
200 @Override
201 public void sync(long txid) {
202 sync();
205 @Override
206 public Long startCacheFlush(final byte[] encodedRegionName, Map<byte[], Long>
207 flushedFamilyNamesToSeq) {
208 return startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq.keySet());
211 @Override
212 public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> flushedFamilyNames) {
213 if (closed.get()) return null;
214 return HConstants.NO_SEQNUM;
217 @Override
218 public void completeCacheFlush(final byte[] encodedRegionName) {
221 @Override
222 public void abortCacheFlush(byte[] encodedRegionName) {
225 @Override
226 public WALCoprocessorHost getCoprocessorHost() {
227 return coprocessorHost;
230 @Override
231 public long getEarliestMemStoreSeqNum(byte[] encodedRegionName) {
232 return HConstants.NO_SEQNUM;
235 @Override
236 public long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
237 return HConstants.NO_SEQNUM;
240 @Override
241 public String toString() {
242 return "WAL disabled.";
245 @Override
246 public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
247 return OptionalLong.empty();
251 @Override
252 public long getNumLogFiles() {
253 return 0;
256 @Override
257 public long getLogFileSize() {
258 return 0;
261 @Override
262 public void addWALActionsListener(WALActionsListener listener) {
263 disabled.registerWALActionsListener(listener);