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.replication.regionserver;
019
020import java.io.EOFException;
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.List;
024import java.util.concurrent.locks.Lock;
025
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.CellScanner;
030import org.apache.hadoop.hbase.CellUtil;
031import org.apache.hadoop.hbase.executor.EventType;
032import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
033import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
034import org.apache.hadoop.hbase.regionserver.HRegionServer;
035import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
036import org.apache.hadoop.hbase.util.FSUtils;
037import org.apache.hadoop.hbase.util.KeyLocker;
038import org.apache.hadoop.hbase.util.Pair;
039import org.apache.hadoop.hbase.wal.WAL.Entry;
040import org.apache.hadoop.hbase.wal.WAL.Reader;
041import org.apache.hadoop.hbase.wal.WALEdit;
042import org.apache.hadoop.hbase.wal.WALFactory;
043import org.apache.yetus.audience.InterfaceAudience;
044import org.slf4j.Logger;
045import org.slf4j.LoggerFactory;
046
047import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
048
049import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
050import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
051import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter;
052
053/**
054 * This callable executed at RS side to replay sync replication wal.
055 */
056@InterfaceAudience.Private
057public class ReplaySyncReplicationWALCallable implements RSProcedureCallable {
058
059  private static final Logger LOG = LoggerFactory.getLogger(ReplaySyncReplicationWALCallable.class);
060
061  private static final String REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE =
062      "hbase.replay.sync.replication.wal.batch.size";
063
064  private static final long DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE = 8 * 1024 * 1024;
065
066  private HRegionServer rs;
067
068  private FileSystem fs;
069
070  private Configuration conf;
071
072  private String peerId;
073
074  private List<String> wals = new ArrayList<>();
075
076  private Exception initError;
077
078  private long batchSize;
079
080  private final KeyLocker<String> peersLock = new KeyLocker<>();
081
082  @Override
083  public Void call() throws Exception {
084    if (initError != null) {
085      throw initError;
086    }
087    LOG.info("Received a replay sync replication wals {} event, peerId={}", wals, peerId);
088    if (rs.getReplicationSinkService() != null) {
089      Lock peerLock = peersLock.acquireLock(wals.get(0));
090      try {
091        for (String wal : wals) {
092          replayWAL(wal);
093        }
094      } finally {
095        peerLock.unlock();
096      }
097    }
098    return null;
099  }
100
101  @Override
102  public void init(byte[] parameter, HRegionServer rs) {
103    this.rs = rs;
104    this.fs = rs.getWALFileSystem();
105    this.conf = rs.getConfiguration();
106    try {
107      ReplaySyncReplicationWALParameter param =
108          ReplaySyncReplicationWALParameter.parseFrom(parameter);
109      this.peerId = param.getPeerId();
110      param.getWalList().forEach(this.wals::add);
111      this.batchSize = rs.getConfiguration().getLong(REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE,
112        DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE);
113    } catch (InvalidProtocolBufferException e) {
114      initError = e;
115    }
116  }
117
118  @Override
119  public EventType getEventType() {
120    return EventType.RS_REPLAY_SYNC_REPLICATION_WAL;
121  }
122
123  private void replayWAL(String wal) throws IOException {
124    try (Reader reader = getReader(wal)) {
125      List<Entry> entries = readWALEntries(reader);
126      while (!entries.isEmpty()) {
127        Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> pair = ReplicationProtbufUtil
128            .buildReplicateWALEntryRequest(entries.toArray(new Entry[entries.size()]));
129        ReplicateWALEntryRequest request = pair.getFirst();
130        rs.getReplicationSinkService().replicateLogEntries(request.getEntryList(),
131            pair.getSecond(), request.getReplicationClusterId(),
132            request.getSourceBaseNamespaceDirPath(), request.getSourceHFileArchiveDirPath());
133        // Read next entries.
134        entries = readWALEntries(reader);
135      }
136    }
137  }
138
139  private Reader getReader(String wal) throws IOException {
140    Path path = new Path(rs.getWALRootDir(), wal);
141    long length = rs.getWALFileSystem().getFileStatus(path).getLen();
142    try {
143      FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf);
144      return WALFactory.createReader(rs.getWALFileSystem(), path, rs.getConfiguration());
145    } catch (EOFException e) {
146      if (length <= 0) {
147        LOG.warn("File is empty. Could not open {} for reading because {}", path, e);
148        return null;
149      }
150      throw e;
151    }
152  }
153
154  // return whether we should include this entry.
155  private boolean filter(Entry entry) {
156    WALEdit edit = entry.getEdit();
157    WALUtil.filterCells(edit, c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY) ? null : c);
158    return !edit.isEmpty();
159  }
160
161  private List<Entry> readWALEntries(Reader reader) throws IOException {
162    List<Entry> entries = new ArrayList<>();
163    if (reader == null) {
164      return entries;
165    }
166    long size = 0;
167    for (;;) {
168      Entry entry = reader.next();
169      if (entry == null) {
170        break;
171      }
172      if (filter(entry)) {
173        entries.add(entry);
174        size += entry.getEdit().heapSize();
175        if (size > batchSize) {
176          break;
177        }
178      }
179    }
180    return entries;
181  }
182}