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
;
24 import java
.util
.OptionalLong
;
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.
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);
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);
70 public List
<WAL
> getWALs() {
71 List
<WAL
> wals
= new ArrayList
<>(1);
77 public WAL
getWAL(RegionInfo region
) throws IOException
{
82 public void close() throws IOException
{
87 public void shutdown() throws IOException
{
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
);
101 if (null != listeners
) {
102 for(WALActionsListener listener
: listeners
) {
103 registerWALActionsListener(listener
);
109 public void registerWALActionsListener(final WALActionsListener listener
) {
110 listeners
.add(listener
);
114 public boolean unregisterWALActionsListener(final WALActionsListener listener
) {
115 return listeners
.remove(listener
);
119 public byte[][] rollWriter() {
120 if (!listeners
.isEmpty()) {
121 for (WALActionsListener listener
: listeners
) {
122 listener
.logRollRequested(false);
124 for (WALActionsListener listener
: listeners
) {
126 listener
.preLogRoll(path
, path
);
127 } catch (IOException exception
) {
128 LOG
.debug("Ignoring exception from listener.", exception
);
131 for (WALActionsListener listener
: listeners
) {
133 listener
.postLogRoll(path
, path
);
134 } catch (IOException exception
) {
135 LOG
.debug("Ignoring exception from listener.", exception
);
143 public byte[][] rollWriter(boolean force
) {
148 public void shutdown() {
149 if(closed
.compareAndSet(false, true)) {
150 if (!this.listeners
.isEmpty()) {
151 for (WALActionsListener listener
: this.listeners
) {
152 listener
.logCloseRequested();
159 public void close() {
164 public long append(RegionInfo info
, WALKeyImpl key
, WALEdit edits
, boolean inMemstore
)
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();
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
);
188 public void updateStore(byte[] encodedRegionName
, byte[] familyName
,
189 Long sequenceid
, boolean onlyIfGreater
) { return; }
193 if (!this.listeners
.isEmpty()) {
194 for (WALActionsListener listener
: this.listeners
) {
195 listener
.postSync(0L, 0);
201 public void sync(long txid
) {
206 public Long
startCacheFlush(final byte[] encodedRegionName
, Map
<byte[], Long
>
207 flushedFamilyNamesToSeq
) {
208 return startCacheFlush(encodedRegionName
, flushedFamilyNamesToSeq
.keySet());
212 public Long
startCacheFlush(final byte[] encodedRegionName
, Set
<byte[]> flushedFamilyNames
) {
213 if (closed
.get()) return null;
214 return HConstants
.NO_SEQNUM
;
218 public void completeCacheFlush(final byte[] encodedRegionName
) {
222 public void abortCacheFlush(byte[] encodedRegionName
) {
226 public WALCoprocessorHost
getCoprocessorHost() {
227 return coprocessorHost
;
231 public long getEarliestMemStoreSeqNum(byte[] encodedRegionName
) {
232 return HConstants
.NO_SEQNUM
;
236 public long getEarliestMemStoreSeqNum(byte[] encodedRegionName
, byte[] familyName
) {
237 return HConstants
.NO_SEQNUM
;
241 public String
toString() {
242 return "WAL disabled.";
246 public OptionalLong
getLogFileSizeIfBeingWritten(Path path
) {
247 return OptionalLong
.empty();
252 public long getNumLogFiles() {
257 public long getLogFileSize() {
262 public void addWALActionsListener(WALActionsListener listener
) {
263 disabled
.registerWALActionsListener(listener
);