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.wal;
019
020import java.io.Closeable;
021import java.io.IOException;
022import java.util.List;
023import java.util.OptionalLong;
024import java.util.concurrent.CompletableFuture;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.hbase.Abortable;
027import org.apache.hadoop.hbase.client.RegionInfo;
028import org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL;
029import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
030import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
031import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
032import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
033import org.apache.yetus.audience.InterfaceAudience;
034
035/**
036 * The Write Ahead Log (WAL) stores all durable edits to the HRegion. This interface provides the
037 * entry point for all WAL implementors.
038 * <p>
039 * See {@link FSHLogProvider} for an example implementation. A single WALProvider will be used for
040 * retrieving multiple WALs in a particular region server and must be threadsafe.
041 */
042@InterfaceAudience.Private
043public interface WALProvider {
044
045  /**
046   * Set up the provider to create wals. will only be called once per instance.
047   * @param factory    factory that made us may not be null
048   * @param conf       may not be null
049   * @param providerId differentiate between providers from one factory. may be null
050   */
051  void init(WALFactory factory, Configuration conf, String providerId, Abortable server)
052    throws IOException;
053
054  /**
055   * @param region the region which we want to get a WAL for it. Could be null.
056   * @return a WAL for writing entries for the given region.
057   */
058  WAL getWAL(RegionInfo region) throws IOException;
059
060  /** Returns the List of WALs that are used by this server */
061  List<WAL> getWALs();
062
063  /**
064   * persist outstanding WALs to storage and stop accepting new appends. This method serves as
065   * shorthand for sending a sync to every WAL provided by a given implementation. Those WALs will
066   * also stop accepting new writes.
067   */
068  void shutdown() throws IOException;
069
070  /**
071   * shutdown utstanding WALs and clean up any persisted state. Call this method only when you will
072   * not need to replay any of the edits to the WALs from this provider. After this call completes,
073   * the underlying resources should have been reclaimed.
074   */
075  void close() throws IOException;
076
077  interface WriterBase extends Closeable {
078    long getLength();
079
080    /**
081     * NOTE: We add this method for {@link WALFileLengthProvider} used for replication, considering
082     * the case if we use {@link AsyncFSWAL},we write to 3 DNs concurrently, according to the
083     * visibility guarantee of HDFS, the data will be available immediately when arriving at DN
084     * since all the DNs will be considered as the last one in pipeline. This means replication may
085     * read uncommitted data and replicate it to the remote cluster and cause data inconsistency.
086     * The method {@link WriterBase#getLength} may return length which just in hdfs client buffer
087     * and not successfully synced to HDFS, so we use this method to return the length successfully
088     * synced to HDFS and replication thread could only read writing WAL file limited by this
089     * length. see also HBASE-14004 and this document for more details:
090     * https://docs.google.com/document/d/11AyWtGhItQs6vsLRIx32PwTxmBY3libXwGXI25obVEY/edit#
091     * @return byteSize successfully synced to underlying filesystem.
092     */
093    long getSyncedLength();
094  }
095
096  // Writers are used internally. Users outside of the WAL should be relying on the
097  // interface provided by WAL.
098  interface Writer extends WriterBase {
099    void sync(boolean forceSync) throws IOException;
100
101    void append(WAL.Entry entry) throws IOException;
102  }
103
104  interface AsyncWriter extends WriterBase {
105    CompletableFuture<Long> sync(boolean forceSync);
106
107    void append(WAL.Entry entry);
108  }
109
110  /**
111   * Get number of the log files this provider is managing
112   */
113  long getNumLogFiles();
114
115  /**
116   * Get size of the log files this provider is managing
117   */
118  long getLogFileSize();
119
120  /**
121   * Add a {@link WALActionsListener}.
122   * <p>
123   * Notice that you must call this method before calling {@link #getWAL(RegionInfo)} as this method
124   * will not effect the {@link WAL} which has already been created. And as long as we can only it
125   * when initialization, it is not thread safe.
126   */
127  void addWALActionsListener(WALActionsListener listener);
128
129  default WALFileLengthProvider getWALFileLengthProvider() {
130    return path -> getWALs().stream().map(w -> w.getLogFileSizeIfBeingWritten(path))
131      .filter(o -> o.isPresent()).findAny().orElse(OptionalLong.empty());
132  }
133
134  // sync replication related
135  default PeerActionListener getPeerActionListener() {
136    return PeerActionListener.DUMMY;
137  }
138
139  default void setSyncReplicationPeerInfoProvider(SyncReplicationPeerInfoProvider provider) {
140  }
141}